From 3b2eb7263730cbe821fae80ccaeb6bfc5a9d869a Mon Sep 17 00:00:00 2001 From: zhennzhang Date: Tue, 27 Dec 2022 11:30:30 +0800 Subject: [PATCH 1/6] [EBAY-KYLIN-3788] Add root table alias for model in kylin5.0 --- .../org/apache/kylin/metadata/model/NDataModel.java | 13 +++++++++---- 1 file changed, 9 insertions(+), 4 deletions(-) diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java index e8df524f381..8649731320c 100644 --- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java +++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java @@ -43,6 +43,7 @@ import java.util.function.Predicate; import java.util.stream.Collectors; +import com.google.common.base.Strings; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.kylin.common.KylinConfig; @@ -402,6 +403,7 @@ public NDataModel(NDataModel other) { this.owner = other.owner; this.description = other.description; this.rootFactTableName = other.rootFactTableName; + this.rootFactTableAlias = other.rootFactTableAlias; this.joinTables = other.joinTables; this.filterCondition = other.filterCondition; this.partitionDesc = other.partitionDesc; @@ -609,8 +611,9 @@ public TblColRef findColumn(String column) throws IllegalArgumentException { column = column.toUpperCase(Locale.ROOT); int cut = column.lastIndexOf('.'); if (cut > 0) { + String table = this.findTable(column.substring(0, cut)).getAlias(); // table specified - result = findColumn(column.substring(0, cut), column.substring(cut + 1)); + result = findColumn(table, column.substring(cut + 1)); } else { // table not specified, try each table for (TableRef tableRef : allTableRefs) { @@ -730,7 +733,10 @@ private void initTableAlias(Map tables) { throw new IllegalStateException("Root fact table does not exist:" + rootFactTableName); TableDesc rootDesc = tables.get(rootFactTableName); - rootFactTableRef = new TableRef(this, rootDesc.getName(), rootDesc, false); + if (Strings.isNullOrEmpty(rootFactTableAlias)) { + rootFactTableAlias = rootDesc.getName(); + } + rootFactTableRef = new TableRef(this, rootFactTableAlias, rootDesc, false); addAlias(rootFactTableRef); factTableRefs.add(rootFactTableRef); @@ -953,8 +959,7 @@ private void reorderJoins(Map tables) { int orderedIndex = 0; Queue joinTableBuff = new ArrayDeque<>(); - TableDesc rootDesc = tables.get(rootFactTableName); - joinTableBuff.addAll(fkMap.get(rootDesc.getName())); + joinTableBuff.addAll(fkMap.get(rootFactTableAlias)); while (!joinTableBuff.isEmpty()) { JoinTableDesc head = joinTableBuff.poll(); orderedJoinTables.set(orderedIndex++, head); From bb56822e49ef74de9c1fefc2f48ae561ee669d3b Mon Sep 17 00:00:00 2001 From: zhennzhang Date: Sun, 8 Jan 2023 16:52:03 +0800 Subject: [PATCH 2/6] [EBAY-KYLIN-3788] fix add root table alias for model in kylin5.0 --- .../org/apache/kylin/metadata/model/NDataModel.java | 10 ++++------ 1 file changed, 4 insertions(+), 6 deletions(-) diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java index 8649731320c..b83b5f63186 100644 --- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java +++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java @@ -43,7 +43,6 @@ import java.util.function.Predicate; import java.util.stream.Collectors; -import com.google.common.base.Strings; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang3.StringUtils; import org.apache.kylin.common.KylinConfig; @@ -611,9 +610,8 @@ public TblColRef findColumn(String column) throws IllegalArgumentException { column = column.toUpperCase(Locale.ROOT); int cut = column.lastIndexOf('.'); if (cut > 0) { - String table = this.findTable(column.substring(0, cut)).getAlias(); // table specified - result = findColumn(table, column.substring(cut + 1)); + result = findColumn(column.substring(0, cut), column.substring(cut + 1)); } else { // table not specified, try each table for (TableRef tableRef : allTableRefs) { @@ -698,7 +696,7 @@ public void init(KylinConfig config) { initJoinTablesForUpgrade(); initTableAlias(tables); initJoinColumns(); - reorderJoins(tables); + reorderJoins(); initJoinsGraph(); initPartitionDesc(); initMultiPartition(); @@ -733,7 +731,7 @@ private void initTableAlias(Map tables) { throw new IllegalStateException("Root fact table does not exist:" + rootFactTableName); TableDesc rootDesc = tables.get(rootFactTableName); - if (Strings.isNullOrEmpty(rootFactTableAlias)) { + if (StringUtils.isEmpty(rootFactTableAlias)) { rootFactTableAlias = rootDesc.getName(); } rootFactTableRef = new TableRef(this, rootFactTableAlias, rootDesc, false); @@ -937,7 +935,7 @@ private void initJoinsGraph() { joinsGraph = new JoinsGraph(rootFactTableRef, joins); } - private void reorderJoins(Map tables) { + private void reorderJoins() { if (CollectionUtils.isEmpty(joinTables)) { return; } From b2179e608e86a41399c48f85d2b52d123aaa42a8 Mon Sep 17 00:00:00 2001 From: zhennzhang Date: Sun, 8 Jan 2023 18:05:31 +0800 Subject: [PATCH 3/6] [EBAY-KYLIN-3788] add table mapping updater and an api for table name update --- .../update/TableSchemaUpdateMapping.java | 72 ++++++++ .../service/update/TableSchemaUpdater.java | 160 ++++++++++++++++++ .../metadata/cube/model/NDataSegment.java | 12 ++ .../kylin/metadata/model/JoinTableDesc.java | 11 ++ .../metadata/model/NTableMetadataManager.java | 13 ++ .../rest/request/TableUpdateRequest.java | 54 ++++++ .../kylin/rest/service/TableService.java | 114 +++++++++++++ .../rest/controller/NTableController.java | 19 +++ 8 files changed, 455 insertions(+) create mode 100644 src/common-service/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdateMapping.java create mode 100644 src/common-service/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdater.java create mode 100644 src/datasource-service/src/main/java/org/apache/kylin/rest/request/TableUpdateRequest.java diff --git a/src/common-service/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdateMapping.java b/src/common-service/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdateMapping.java new file mode 100644 index 00000000000..1b4d4a6c363 --- /dev/null +++ b/src/common-service/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdateMapping.java @@ -0,0 +1,72 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.rest.service.update; + +import java.io.Serializable; +import java.util.Locale; + +import com.google.common.base.Preconditions; +import com.google.common.base.Strings; + +public class TableSchemaUpdateMapping implements Serializable { + + private String database; + + private String tableName; + + public boolean isDatabaseChanged() { + return !Strings.isNullOrEmpty(database); + } + + public String getDatabase(String dbName) { + String ret = isDatabaseChanged() ? database : dbName; + return ret.toUpperCase(Locale.ROOT); + } + + public void setDatabase(String database) { + this.database = database; + } + + public boolean isTableNameChanged() { + return !Strings.isNullOrEmpty(tableName); + } + + public String getTableName(String tblName) { + String ret = isTableNameChanged() ? tableName : tblName; + return ret.toUpperCase(Locale.ROOT); + } + + public void setTableName(String tableName) { + this.tableName = tableName; + } + + public boolean isTableIdentityChanged() { + return isDatabaseChanged() || isTableNameChanged(); + } + + public String getTableIdentity(String tableIdentity) { + String[] tableNameEs = tableIdentity.split("\\."); + Preconditions.checkArgument(tableNameEs.length == 2); + return getTableIdentity(tableNameEs[0], tableNameEs[1]); + } + + public String getTableIdentity(String database, String tableName) { + return getDatabase(database) + "." + getTableName(tableName); + } +} \ No newline at end of file diff --git a/src/common-service/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdater.java b/src/common-service/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdater.java new file mode 100644 index 00000000000..16791a6cafc --- /dev/null +++ b/src/common-service/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdater.java @@ -0,0 +1,160 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.rest.service.update; + +import java.util.ArrayList; +import java.util.List; +import java.util.Locale; +import java.util.Map; + +import org.apache.kylin.common.KylinConfig; +import org.apache.kylin.metadata.cube.model.NDataSegment; +import org.apache.kylin.metadata.cube.model.NDataflow; +import org.apache.kylin.metadata.cube.model.NDataflowManager; +import org.apache.kylin.metadata.model.JoinTableDesc; +import org.apache.kylin.metadata.model.NDataModel; +import org.apache.kylin.metadata.model.NDataModelManager; +import org.apache.kylin.metadata.model.Segments; +import org.apache.kylin.metadata.model.TableDesc; + +import com.google.common.base.Strings; +import com.google.common.collect.Maps; + +public class TableSchemaUpdater { + + public static TableDesc dealWithMappingForTable(TableDesc other, Map mappings) { + TableSchemaUpdateMapping mapping = getTableSchemaUpdateMapping(mappings, other.getIdentity()); + if (mapping == null) { + return other; + } + + TableDesc copy = new TableDesc(other); + + copy.setDatabase(mapping.getDatabase(other.getDatabase())); + copy.setName(mapping.getTableName(other.getName())); + // It will always be a new one + copy.setLastModified(0L); + + return copy; + } + + public static NDataModel dealWithMappingForModel(KylinConfig config, String project, NDataModel other, + Map mappings) { + + // Currently, model with filter condition is not supported. + if (!Strings.isNullOrEmpty(other.getFilterCondition())) { + throw new UnsupportedOperationException("Cannot deal with filter condition " + other.getFilterCondition()); + } + + // Currently, model with computed columns is not supported + if (other.getComputedColumnDescs().size() != 0) { + throw new UnsupportedOperationException( + "Cannot deal with filter condition " + other.getComputedColumnDescs()); + } + NDataModel copy = NDataModelManager.getInstance(config, project).copyForWrite(other); + + // mapping for root fact table identity + TableSchemaUpdateMapping rootMapping = getTableSchemaUpdateMapping(mappings, other.getRootFactTableName()); + if (rootMapping != null) { + TableDesc rootFactTable = other.getRootFactTable().getTableDesc(); + copy.setRootFactTableName( + rootMapping.getTableIdentity(rootFactTable.getDatabase(), rootFactTable.getName())); + } + + // mapping for join tables + List joinTables = other.getJoinTables(); + List joinTablesCopy = new ArrayList<>(joinTables.size()); + for (int i = 0; i < joinTables.size(); i++) { + JoinTableDesc joinTable = joinTables.get(i); + joinTablesCopy.add(JoinTableDesc.getCopyOf(joinTable)); + String tableIdentity = joinTable.getTable(); + TableSchemaUpdateMapping mapping = getTableSchemaUpdateMapping(mappings, tableIdentity); + if (mapping != null && mapping.isTableIdentityChanged()) { + joinTablesCopy.get(i).setTable(mapping.getTableIdentity(tableIdentity)); + } + } + copy.setJoinTables(joinTablesCopy); + + return copy; + } + + public static NDataflow dealWithMappingForDataFlow(KylinConfig config, String project, NDataflow other, Map mappings) { + + NDataflow copy = NDataflowManager.getInstance(config, project).copy(other); + copy.setLastModified(other.getLastModified()); + + // mapping for segments + if (other.getSegments() != null && !other.getSegments().isEmpty()) { + Segments segmentsCopy = new Segments<>(); + for (NDataSegment segment : other.getSegments()) { + NDataSegment segmentCopy = new NDataSegment(segment); + segmentCopy.setDataflow(copy); + // mapping for snapshot + Map snapshotCopy = + replaceTableIdentityForTableSnapshots(segment.getSnapshots(), mappings); + // mapping for column source bytes + Map columnSourceBytesCopy = + replaceColumnIdentityForColumnSourceBytes(segment.getColumnSourceBytes(), mappings); + + segmentCopy.setSnapshots(snapshotCopy); + segmentCopy.setColumnSourceBytes(columnSourceBytesCopy); + + segmentsCopy.add(segmentCopy); + } + copy.setSegments(segmentsCopy); + } + return copy; + } + + private static Map replaceTableIdentityForTableSnapshots(Map snapshots, + Map mappings) { + Map snapshotsCopy = Maps.newHashMapWithExpectedSize(snapshots.size()); + for (String tableIdentity : snapshots.keySet()) { + String resPath = snapshots.get(tableIdentity); + TableSchemaUpdateMapping mapping = getTableSchemaUpdateMapping(mappings, tableIdentity); + if (mapping != null && mapping.isTableIdentityChanged()) { + tableIdentity = mapping.getTableIdentity(tableIdentity); + } + snapshotsCopy.put(tableIdentity, resPath); + } + return snapshotsCopy; + } + + private static Map replaceColumnIdentityForColumnSourceBytes(Map columnSourceBytes, + Map mappings) { + Map copy = Maps.newHashMapWithExpectedSize(columnSourceBytes.size()); + for (String columnIdentity : columnSourceBytes.keySet()) { + Long bytes = columnSourceBytes.get(columnIdentity); + String tableIdentity = columnIdentity.substring(0, columnIdentity.lastIndexOf(".")); + String columnName = columnIdentity.substring(columnIdentity.lastIndexOf(".")); + TableSchemaUpdateMapping mapping = getTableSchemaUpdateMapping(mappings, tableIdentity); + if (mapping != null && mapping.isTableIdentityChanged()) { + tableIdentity = mapping.getTableIdentity(tableIdentity); + columnIdentity = tableIdentity + columnName; + } + copy.put(columnIdentity, bytes); + } + return copy; + } + + public static TableSchemaUpdateMapping getTableSchemaUpdateMapping(Map mappings, + String key) { + return mappings.get(key.toUpperCase(Locale.ROOT)); + } +} diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataSegment.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataSegment.java index a3abc92b37d..9311eb06db0 100644 --- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataSegment.java +++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataSegment.java @@ -506,6 +506,13 @@ public Map getDictionaries() { return isCachedAndShared() ? ImmutableMap.copyOf(dictionaries) : dictionaries; } + public Map getSnapshots() { + if (snapshots == null) + snapshots = Maps.newConcurrentMap(); + + return isCachedAndShared() ? ImmutableMap.copyOf(snapshots) : snapshots; + } + public void putDictResPath(TblColRef col, String dictResPath) { checkIsNotCachedAndShared(); getDictionaries(); // touch to create @@ -518,6 +525,11 @@ public void setDictionaries(Map dictionaries) { this.dictionaries = dictionaries; } + public void setSnapshots(Map snapshots) { + checkIsNotCachedAndShared(); + this.snapshots = snapshots; + } + public long getCreateTimeUTC() { return createTimeUTC; } diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinTableDesc.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinTableDesc.java index 169a79cac58..9aa7005f9ca 100644 --- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinTableDesc.java +++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/JoinTableDesc.java @@ -80,6 +80,17 @@ public boolean hasDifferentAntiFlattenable(JoinTableDesc other) { return this.isFlattenable() ^ other.isFlattenable(); } + public static JoinTableDesc getCopyOf(JoinTableDesc other) { + JoinTableDesc copy = new JoinTableDesc(); + copy.table = other.table; + copy.kind = other.kind; + copy.alias = other.alias; + copy.join = other.join; + copy.flattenable = other.flattenable; + copy.joinRelationTypeEnum = other.joinRelationTypeEnum; + return copy; + } + @Override public boolean equals(Object o) { if (this == o) diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NTableMetadataManager.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NTableMetadataManager.java index 05a28bca36f..ac24b0b45d9 100644 --- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NTableMetadataManager.java +++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NTableMetadataManager.java @@ -173,6 +173,19 @@ public void removeSourceTable(String tableIdentity) { srcTableCrud.delete(t); } + public void saveNewTableExtFromOld(String oldTableId, String prj, String newTableId) throws IOException { + String path = TableExtDesc.concatRawResourcePath(oldTableId); + ResourceStore store = getStore(); + TableExtDesc newTableExt = store.getResource(path, TABLE_EXT_SERIALIZER); + if (newTableExt != null) { + newTableExt.setIdentity(newTableId); + newTableExt.setLastModified(0L); + + newTableExt.init(prj); + srcExtCrud.save(newTableExt); + } + } + /** * 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 diff --git a/src/datasource-service/src/main/java/org/apache/kylin/rest/request/TableUpdateRequest.java b/src/datasource-service/src/main/java/org/apache/kylin/rest/request/TableUpdateRequest.java new file mode 100644 index 00000000000..8576dc80732 --- /dev/null +++ b/src/datasource-service/src/main/java/org/apache/kylin/rest/request/TableUpdateRequest.java @@ -0,0 +1,54 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ +package org.apache.kylin.rest.request; + +import java.io.Serializable; +import java.util.Map; +import java.util.Set; + +import org.apache.kylin.rest.service.update.TableSchemaUpdateMapping; + +public class TableUpdateRequest implements Serializable { + private Map mapping; + private Set cubeSetToAffect; + private boolean isUseExisting; + + public Map getMapping() { + return mapping; + } + + public void setMapping(Map mapping) { + this.mapping = mapping; + } + + public Set getCubeSetToAffect() { + return cubeSetToAffect; + } + + public void setCubeSetToAffect(Set cubeSetToAffect) { + this.cubeSetToAffect = cubeSetToAffect; + } + + public boolean isUseExisting() { + return isUseExisting; + } + + public void setIsUseExisting(boolean useExisting) { + isUseExisting = useExisting; + } +} diff --git a/src/datasource-service/src/main/java/org/apache/kylin/rest/service/TableService.java b/src/datasource-service/src/main/java/org/apache/kylin/rest/service/TableService.java index f4c94be6843..6469883d57e 100644 --- a/src/datasource-service/src/main/java/org/apache/kylin/rest/service/TableService.java +++ b/src/datasource-service/src/main/java/org/apache/kylin/rest/service/TableService.java @@ -96,7 +96,9 @@ import org.apache.kylin.metadata.cube.model.NDataLoadingRange; import org.apache.kylin.metadata.cube.model.NDataLoadingRangeManager; import org.apache.kylin.metadata.cube.model.NDataSegment; +import org.apache.kylin.metadata.cube.model.NDataflow; import org.apache.kylin.metadata.cube.model.NDataflowManager; +import org.apache.kylin.metadata.cube.model.NDataflowUpdate; import org.apache.kylin.metadata.cube.model.NIndexPlanManager; import org.apache.kylin.metadata.cube.model.NSegmentConfigHelper; import org.apache.kylin.metadata.datatype.DataType; @@ -135,6 +137,7 @@ import org.apache.kylin.rest.aspect.Transaction; import org.apache.kylin.rest.cluster.ClusterManager; import org.apache.kylin.rest.constant.JobInfoEnum; +import org.apache.kylin.rest.exception.BadRequestException; import org.apache.kylin.rest.request.AutoMergeRequest; import org.apache.kylin.rest.request.DateRangeRequest; import org.apache.kylin.rest.request.S3TableExtInfo; @@ -153,6 +156,8 @@ import org.apache.kylin.rest.response.TableRefreshAll; import org.apache.kylin.rest.response.TablesAndColumnsResponse; import org.apache.kylin.rest.security.KerberosLoginManager; +import org.apache.kylin.rest.service.update.TableSchemaUpdateMapping; +import org.apache.kylin.rest.service.update.TableSchemaUpdater; import org.apache.kylin.rest.source.DataSourceState; import org.apache.kylin.rest.util.AclEvaluate; import org.apache.kylin.rest.util.AclPermissionUtil; @@ -2024,4 +2029,113 @@ public List getTablesOfModel(String project, String modelAlias) { public TableExtDesc getOrCreateTableExt(String project, TableDesc t) { return getManager(NTableMetadataManager.class, project).getOrCreateTableExt(t); } + + /** + * 1. Check whether it's able to do the change + * - related model should be offline + * 2. Get all influenced metadata + * - table + * - model + * - dataflow + * 3. Update the metadata + * 4. Save the updated metadata + * - table + * - model + * - dataflow + * 5. Delete dirty table metadata + */ + public void updateHiveTable(String projectName, Map mapping0, Set modelSetToAffect, boolean isUseExisting) throws IOException { + final ProjectInstance prjInstance = getProjectManager().getProject(projectName); + val kylinConfig = KylinConfig.getInstanceFromEnv(); + if (prjInstance == null) { + throw new BadRequestException("Project " + projectName + " does not exist"); + } + // To deal with case sensitive issue for table resource path + final String project = prjInstance.getName(); + aclEvaluate.checkProjectWritePermission(project); + + boolean ifAllModelUpdate = true; + + // Normalize mapping + Map mapping = mapping0.entrySet().stream().collect(Collectors.toMap(entry -> entry.getKey().toUpperCase(Locale.ROOT), Map.Entry::getValue)); + + // Check whether it's able to do the change + Set infModels = NDataModelManager.getInstance(kylinConfig, project) + .listAllModels().stream().filter(model -> isTablesUsed(model, mapping.keySet())).collect(Collectors.toSet()); + if (modelSetToAffect != null && !modelSetToAffect.isEmpty()) { + int nModel = infModels.size(); + final Set modelSetToInf = modelSetToAffect.stream().map(model -> model.toUpperCase(Locale.ROOT)).collect(Collectors.toSet()); + infModels = infModels.stream().filter(model -> modelSetToInf.contains(model.getAlias().toUpperCase(Locale.ROOT))).collect(Collectors.toSet()); + ifAllModelUpdate = nModel == infModels.size(); + } + // Currently, only only model with offline status is supported to update with mappings. + Set readyModelSet = infModels.stream().filter(model -> NDataflowManager.getInstance(kylinConfig, project) + .getDataflowByModelAlias(model.getAlias()).getStatus() == RealizationStatusEnum.OFFLINE) + .collect(Collectors.toSet()); + // At least 1 model should be update here, otherwise it will throw BadRequestException. + if (!readyModelSet.isEmpty()) { + throw new BadRequestException("Influenced models " + readyModelSet + " should be OFFLINE"); + } + logger.info("Influenced cubes {}", infModels); + + // Get influenced metadata and update the metadata + NTableMetadataManager tableManager = NTableMetadataManager.getInstance(kylinConfig, project); + // -- 1. table + Map newTables = mapping.keySet().stream() + .map(tableManager::getTableDesc).collect(Collectors.toMap(TableDesc::getIdentity, + t -> TableSchemaUpdater.dealWithMappingForTable(t, mapping))); + Map existingTables = newTables.entrySet().stream() + .filter(t -> tableManager.getTableDesc(t.getValue().getIdentity()) != null) + .collect(Collectors.toMap(Map.Entry::getKey, t -> t.getValue().getIdentity())); + if (!existingTables.isEmpty()) { + if (isUseExisting) { + logger.info("Will use existing tables {}", existingTables.values()); + } else { + throw new BadRequestException("Tables " + existingTables.values() + " already exist"); + } + } + // -- 2. model + Map newModels = infModels.stream().map(model -> TableSchemaUpdater + .dealWithMappingForModel(kylinConfig, project, model, mapping)) + .collect(Collectors.toMap(NDataModel::getAlias, model -> model)); + // -- 3. dataflow + Map newDataflow = infModels.stream() + .map(model -> NDataflowManager.getInstance(kylinConfig, project).getDataflowByModelAlias(model.getAlias())) + .map(dataflow -> TableSchemaUpdater.dealWithMappingForDataFlow(kylinConfig, project, dataflow, mapping)) + .collect(Collectors.toMap(NDataflow::resourceName, dataflow -> dataflow)); + + // Save the updated metadata + // -- 1. table & table_ext + for (Map.Entry entry : newTables.entrySet()) { + if (existingTables.containsKey(entry.getKey())) { + continue; + } + NTableMetadataManager.getInstance(kylinConfig, project).saveSourceTable(entry.getValue()); + NTableMetadataManager.getInstance(kylinConfig, project).saveNewTableExtFromOld(entry.getKey(), project, entry.getValue().getIdentity()); + } + // -- 2. model + for (Map.Entry entry : newModels.entrySet()) { + NDataModelManager.getInstance(kylinConfig, project).updateDataModelDesc(entry.getValue()); + } + // -- 3. dataflow + for (Map.Entry entry : newDataflow.entrySet()) { + NDataflowUpdate update = new NDataflowUpdate(entry.getValue().getId()); + NDataflowManager.getInstance(kylinConfig, project).updateDataflow(update); + } + + if (ifAllModelUpdate) { + // Delete dirty table metadata + for (String entry : newTables.keySet()) { + NTableMetadataManager.getInstance(kylinConfig, project).removeTableExt(entry); + NTableMetadataManager.getInstance(kylinConfig, project).removeSourceTable(entry); + } + } + } + + private static boolean isTablesUsed(NDataModel model, Set tables) { + Set usingTables = model.getAllTables().stream().map(TableRef::getTableIdentity) + .collect(Collectors.toSet()); + usingTables.retainAll(tables); + return !usingTables.isEmpty(); + } } diff --git a/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NTableController.java b/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NTableController.java index 6b63490708c..be17ad69fc4 100644 --- a/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NTableController.java +++ b/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NTableController.java @@ -43,12 +43,15 @@ import org.apache.kylin.common.util.StringUtil; import org.apache.kylin.metadata.model.TableDesc; import org.apache.kylin.metadata.project.NProjectManager; +import org.apache.kylin.rest.exception.BadRequestException; +import org.apache.kylin.rest.exception.InternalErrorException; import org.apache.kylin.rest.request.AWSTableLoadRequest; import org.apache.kylin.rest.request.AutoMergeRequest; import org.apache.kylin.rest.request.PartitionKeyRequest; import org.apache.kylin.rest.request.PushDownModeRequest; import org.apache.kylin.rest.request.ReloadTableRequest; import org.apache.kylin.rest.request.TableLoadRequest; +import org.apache.kylin.rest.request.TableUpdateRequest; import org.apache.kylin.rest.request.TopTableRequest; import org.apache.kylin.rest.request.UpdateAWSTableExtDescRequest; import org.apache.kylin.rest.response.AutoMergeConfigResponse; @@ -80,6 +83,7 @@ import org.springframework.web.bind.annotation.PutMapping; import org.springframework.web.bind.annotation.RequestBody; import org.springframework.web.bind.annotation.RequestMapping; +import org.springframework.web.bind.annotation.RequestMethod; import org.springframework.web.bind.annotation.RequestParam; import org.springframework.web.bind.annotation.ResponseBody; @@ -461,6 +465,21 @@ public EnvelopeResponse preReloadTable(@RequestParam(val return new EnvelopeResponse<>(KylinException.CODE_SUCCESS, result, ""); } + @RequestMapping(value = "/{project}/update", method = {RequestMethod.POST}, produces = {"application/json"}) + @ResponseBody + public void updateHiveTables(@PathVariable String project, @RequestBody TableUpdateRequest request) + throws IOException { + try { + tableService.updateHiveTable(project, request.getMapping(), request.getCubeSetToAffect(), request.isUseExisting()); + } catch (BadRequestException e) { + log.error("Failed to update Hive Table", e); + throw e; + } catch (Throwable e) { + log.error("Failed to update Hive Table", e); + throw new InternalErrorException(e.getLocalizedMessage(), e); + } + } + @ApiOperation(value = "reload", tags = { "AI" }) @PostMapping(value = "/reload", produces = { HTTP_VND_APACHE_KYLIN_JSON }) @ResponseBody From 812207477100b0c8d72845853cfc7e6317c7ef2d Mon Sep 17 00:00:00 2001 From: zhennzhang Date: Sun, 8 Jan 2023 18:56:17 +0800 Subject: [PATCH 4/6] [EBAY-KYLIN-3788] add metadata table mapping updater --- .../update/TableSchemaUpdaterTest.java | 135 ++ .../update/TableSchemaUpdateMapping.json | 19 + .../82fa7671-a935-45f5-8779-85703601f49a.json | 13 + .../89af4ee2-2cdb-4b07-b39e-4c29856309aa.json | 59 + .../82fa7671-a935-45f5-8779-85703601f49a.json | 1324 +++++++++++++++++ .../update/default/table/EDW.CAL_DT.json | 414 ++++++ .../update/default/table/TEST.COUNTRY.json | 26 + .../update/default/table/TEST.KYLIN_FACT.json | 86 ++ .../default/table/TEST.TEST_ACCOUNT.json | 41 + .../table/TEST.TEST_CATEGORY_GROUPINGS.json | 156 ++ .../kylin/metadata/cube/model/NDataflow.java | 28 + .../metadata/cube/model/NDataflowManager.java | 6 + .../kylin/metadata/model/NDataModel.java | 62 + .../kylin/metadata/model/SegmentConfig.java | 21 + .../TableSchemaUpdateMapping.json | 19 + 15 files changed, 2409 insertions(+) create mode 100644 src/common-service/src/test/java/org/apache/kylin/rest/service/update/TableSchemaUpdaterTest.java create mode 100644 src/common-service/src/test/resources/update/TableSchemaUpdateMapping.json create mode 100755 src/common-service/src/test/resources/update/default/dataflow/82fa7671-a935-45f5-8779-85703601f49a.json create mode 100755 src/common-service/src/test/resources/update/default/dataflow/89af4ee2-2cdb-4b07-b39e-4c29856309aa.json create mode 100644 src/common-service/src/test/resources/update/default/model_desc/82fa7671-a935-45f5-8779-85703601f49a.json create mode 100644 src/common-service/src/test/resources/update/default/table/EDW.CAL_DT.json create mode 100644 src/common-service/src/test/resources/update/default/table/TEST.COUNTRY.json create mode 100644 src/common-service/src/test/resources/update/default/table/TEST.KYLIN_FACT.json create mode 100644 src/common-service/src/test/resources/update/default/table/TEST.TEST_ACCOUNT.json create mode 100644 src/common-service/src/test/resources/update/default/table/TEST.TEST_CATEGORY_GROUPINGS.json create mode 100644 src/metadata-server/src/test/resources/update_hive_table/TableSchemaUpdateMapping.json diff --git a/src/common-service/src/test/java/org/apache/kylin/rest/service/update/TableSchemaUpdaterTest.java b/src/common-service/src/test/java/org/apache/kylin/rest/service/update/TableSchemaUpdaterTest.java new file mode 100644 index 00000000000..7de777bbcd0 --- /dev/null +++ b/src/common-service/src/test/java/org/apache/kylin/rest/service/update/TableSchemaUpdaterTest.java @@ -0,0 +1,135 @@ +/* + * Licensed to the Apache Software Foundation (ASF) under one + * or more contributor license agreements. See the NOTICE file + * distributed with this work for additional information + * regarding copyright ownership. The ASF licenses this file + * to you under the Apache License, Version 2.0 (the + * "License"); you may not use this file except in compliance + * with the License. You may obtain a copy of the License at + * + * http://www.apache.org/licenses/LICENSE-2.0 + * + * Unless required by applicable law or agreed to in writing, software + * distributed under the License is distributed on an "AS IS" BASIS, + * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied. + * See the License for the specific language governing permissions and + * limitations under the License. + */ + +package org.apache.kylin.rest.service.update; + +import java.io.ByteArrayInputStream; +import java.io.ByteArrayOutputStream; +import java.io.DataInputStream; +import java.io.DataOutputStream; +import java.io.File; +import java.io.FileInputStream; +import java.io.IOException; +import java.nio.charset.StandardCharsets; +import java.nio.file.Files; +import java.util.Map; +import java.util.Set; + +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.JsonUtil; +import org.apache.kylin.common.util.NLocalFileMetadataTestCase; +import org.apache.kylin.metadata.cube.model.NDataflow; +import org.apache.kylin.metadata.cube.model.NDataflowManager; +import org.apache.kylin.metadata.model.NDataModel; +import org.apache.kylin.metadata.model.NDataModelManager; +import org.apache.kylin.metadata.model.NTableMetadataManager; +import org.apache.kylin.metadata.model.TableDesc; +import org.junit.Assert; +import org.junit.Before; +import org.junit.Test; + +import com.fasterxml.jackson.core.type.TypeReference; +import com.google.common.collect.Sets; + +public class TableSchemaUpdaterTest extends NLocalFileMetadataTestCase { + + private final String mappingRootPath = "src/test/resources/update"; + private final String mappingFileName = "TableSchemaUpdateMapping.json"; + private Map mappings; + private static String PROJECT_NAME = "default"; + + @Before + public void setUp() throws IOException { + this.createTestMetadata(); + + File mappingFile = new File(mappingRootPath + "/" + mappingFileName); + String content = new String(Files.readAllBytes(mappingFile.toPath()), StandardCharsets.UTF_8); + mappings = JsonUtil.readValue(content, new TypeReference>() { + }); + } + + @Test + public void testDealWithMappingForTable() throws IOException { + NTableMetadataManager tableMetaManager = NTableMetadataManager.getInstance(getTestConfig(), PROJECT_NAME); + ResourceStore store = tableMetaManager.getStore(); + + Set tables = Sets.newHashSet(); + for (String tableIdentity : mappings.keySet()) { + tables.add(store.getResource(TableDesc.concatResourcePath(tableIdentity, "default"), + NTableMetadataManager.getInstance(getTestConfig(), PROJECT_NAME).getTableMetadataSerializer())); + } + + for (TableDesc tableDesc : tables) { + TableDesc updated = TableSchemaUpdater.dealWithMappingForTable(tableDesc, mappings); + updated = reInit(updated, NTableMetadataManager.getInstance(getTestConfig(), PROJECT_NAME).getTableMetadataSerializer()); + + try (DataInputStream bis = new DataInputStream(new FileInputStream( + new File(mappingRootPath + TableDesc.concatResourcePath(updated.getIdentity(), PROJECT_NAME))))) { + TableDesc expected = NTableMetadataManager.getInstance(getTestConfig(), PROJECT_NAME).getTableMetadataSerializer().deserialize(bis); + Assert.assertEquals(expected, updated); + } catch (Exception e) { + Assert.fail("Table identity is not updated correctly"); + } + } + } + + @Test + public void testDealWithMappingForModel() throws IOException { + NDataModelManager dataModelManager = NDataModelManager.getInstance(getTestConfig(), PROJECT_NAME); + NDataModel model = dataModelManager.getDataModelDescByAlias("ut_inner_join_cube_partial"); + + NDataModel updated = TableSchemaUpdater.dealWithMappingForModel(getTestConfig(), PROJECT_NAME, model, mappings); + updated = reInit(updated, dataModelManager.getDataModelSerializer()); + + try (DataInputStream bis = new DataInputStream( + new FileInputStream(new File(mappingRootPath + NDataModel.concatResourcePath(updated.getUuid(), PROJECT_NAME))))) { + NDataModel expected = dataModelManager.getDataModelSerializer().deserialize(bis); + Assert.assertTrue(expected.equalsRaw(updated)); + } catch (Exception e) { + Assert.fail("Model is not updated correctly"); + } + } + + @Test + public void testDealWithMappingForDataflow() throws IOException { + NDataflowManager dataflowManager = NDataflowManager.getInstance(getTestConfig(), PROJECT_NAME); + NDataflow dataflow = dataflowManager.getDataflow("89af4ee2-2cdb-4b07-b39e-4c29856309aa"); + + NDataflow updated = TableSchemaUpdater.dealWithMappingForDataFlow(getTestConfig(), PROJECT_NAME, dataflow, mappings); + updated = reInit(updated, dataflowManager.getDataflowSerializer()); + + try (DataInputStream bis = new DataInputStream( + new FileInputStream(new File(mappingRootPath + NDataflow.concatResourcePath(updated.getUuid(), PROJECT_NAME))))) { + NDataflow expected = dataflowManager.getDataflowSerializer().deserialize(bis); + Assert.assertTrue(expected.equalsRaw(updated)); + } catch (Exception e) { + Assert.fail("Dataflow is not updated correctly"); + } + } + + private T reInit(T obj, Serializer serializer) throws IOException { + ByteArrayOutputStream baos = new ByteArrayOutputStream(); + DataOutputStream dos = new DataOutputStream(baos); + serializer.serialize(obj, dos); + ByteArrayInputStream bais = new ByteArrayInputStream(baos.toByteArray()); + DataInputStream dis = new DataInputStream(bais); + return serializer.deserialize(dis); + } +} diff --git a/src/common-service/src/test/resources/update/TableSchemaUpdateMapping.json b/src/common-service/src/test/resources/update/TableSchemaUpdateMapping.json new file mode 100644 index 00000000000..bd601c81002 --- /dev/null +++ b/src/common-service/src/test/resources/update/TableSchemaUpdateMapping.json @@ -0,0 +1,19 @@ +{ + "DEFAULT.TEST_KYLIN_FACT": { + "database": "TEST", + "tableName": "KYLIN_FACT" + }, + "DEFAULT.TEST_ACCOUNT": { + "database": "TEST" + }, + "DEFAULT.TEST_COUNTRY": { + "database": "TEST", + "tableName": "COUNTRY" + }, + "EDW.TEST_CAL_DT": { + "tableName": "CAL_DT" + }, + "DEFAULT.TEST_CATEGORY_GROUPINGS": { + "database": "TEST" + } +} \ No newline at end of file diff --git a/src/common-service/src/test/resources/update/default/dataflow/82fa7671-a935-45f5-8779-85703601f49a.json b/src/common-service/src/test/resources/update/default/dataflow/82fa7671-a935-45f5-8779-85703601f49a.json new file mode 100755 index 00000000000..6e765b0e90c --- /dev/null +++ b/src/common-service/src/test/resources/update/default/dataflow/82fa7671-a935-45f5-8779-85703601f49a.json @@ -0,0 +1,13 @@ +{ + "uuid" : "82fa7671-a935-45f5-8779-85703601f49a", + "last_modified" : 1509805113773, + "version" : "2.3.0.20500", + "description" : "this is ut_inner_join_cube_partial", + "owner" : "test_owner", + "mp_values" : null, + "create_time_utc" : 1509791792901, + "status" : "OFFLINE", + "cost" : 50, + "segments" : [], + "storage_location_identifier" : null +} \ No newline at end of file diff --git a/src/common-service/src/test/resources/update/default/dataflow/89af4ee2-2cdb-4b07-b39e-4c29856309aa.json b/src/common-service/src/test/resources/update/default/dataflow/89af4ee2-2cdb-4b07-b39e-4c29856309aa.json new file mode 100755 index 00000000000..5469893e518 --- /dev/null +++ b/src/common-service/src/test/resources/update/default/dataflow/89af4ee2-2cdb-4b07-b39e-4c29856309aa.json @@ -0,0 +1,59 @@ +{ + "uuid" : "89af4ee2-2cdb-4b07-b39e-4c29856309aa", + "last_modified" : 1522545367929, + "version" : "2.3.0.20500", + "description" : "test_description", + "owner" : "test_owner", + "mp_values" : null, + "create_time_utc" : 1509791792901, + "status" : "ONLINE", + "query_hit_count" : 3, + "cost" : 50, + "segments" : [ { + "id" : "ef5e0663-feba-4ed2-b71c-21958122bbff", + "name" : "FULL_BUILD", + "create_time_utc" : 1522545284831, + "status" : "READY", + "segRange" : { + "@class" : "org.apache.kylin.metadata.model.SegmentRange$TimePartitionedSegmentRange", + "date_range_start" : 0, + "date_range_end" : 9223372036854775807 + }, + "timeRange" : null, + "dictionaries" : { + "TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME" : "/default/dict/DEFAULT.TEST_CATEGORY_GROUPINGS/CATEG_LVL2_NAME/5a78447d-1843-4aac-a6ac-04ebd0d36f8a.dict", + "TEST_KYLIN_FACT.LEAF_CATEG_ID" : "/default/dict/DEFAULT.TEST_KYLIN_FACT/LEAF_CATEG_ID/1b3fa298-89fa-4f7a-9e77-fecfe7099710.dict", + "SELLER_COUNTRY.NAME" : "/default/dict/DEFAULT.TEST_COUNTRY/NAME/322c0f21-cdf1-43ec-a4d9-e69fbb32ec25.dict", + "BUYER_COUNTRY.NAME" : "/default/dict/DEFAULT.TEST_COUNTRY/NAME/322c0f21-cdf1-43ec-a4d9-e69fbb32ec25.dict", + "TEST_KYLIN_FACT.TRANS_ID" : "/default/dict/DEFAULT.TEST_KYLIN_FACT/TRANS_ID/965f9840-4917-4345-af44-93d7f8db3732.dict", + "TEST_KYLIN_FACT.TEST_COUNT_DISTINCT_BITMAP" : "/default/dict/DEFAULT.TEST_KYLIN_FACT/TEST_COUNT_DISTINCT_BITMAP/0887f77f-e473-474f-acf0-cbd30b9cd02a.dict", + "SELLER_ACCOUNT.ACCOUNT_BUYER_LEVEL" : "/default/dict/DEFAULT.TEST_ACCOUNT/ACCOUNT_BUYER_LEVEL/6df4cf40-73c6-4f72-8725-3da41bedecb1.dict", + "BUYER_ACCOUNT.ACCOUNT_BUYER_LEVEL" : "/default/dict/DEFAULT.TEST_ACCOUNT/ACCOUNT_BUYER_LEVEL/6df4cf40-73c6-4f72-8725-3da41bedecb1.dict", + "SELLER_ACCOUNT.ACCOUNT_SELLER_LEVEL" : "/default/dict/DEFAULT.TEST_ACCOUNT/ACCOUNT_SELLER_LEVEL/9ea05627-2b99-476e-87df-fa1652b467b2.dict", + "BUYER_ACCOUNT.ACCOUNT_SELLER_LEVEL" : "/default/dict/DEFAULT.TEST_ACCOUNT/ACCOUNT_SELLER_LEVEL/9ea05627-2b99-476e-87df-fa1652b467b2.dict", + "TEST_KYLIN_FACT.SLR_SEGMENT_CD" : "/default/dict/DEFAULT.TEST_KYLIN_FACT/SLR_SEGMENT_CD/b8b66292-0af9-4a3b-82fd-9a2090099f1d.dict", + "TEST_KYLIN_FACT.PRICE" : "/default/dict/DEFAULT.TEST_KYLIN_FACT/PRICE/19d380fa-5fd8-4615-9d86-38b9a8693155.dict", + "TEST_ORDER.BUYER_ID" : "/default/dict/DEFAULT.TEST_ORDER/BUYER_ID/791226e8-bb0c-40f0-85ff-03b5a139dfbd.dict", + "SELLER_ACCOUNT.ACCOUNT_COUNTRY" : "/default/dict/DEFAULT.TEST_ACCOUNT/ACCOUNT_COUNTRY/cd875303-942d-4cfe-86bf-3cfa7dc7309f.dict", + "BUYER_ACCOUNT.ACCOUNT_COUNTRY" : "/default/dict/DEFAULT.TEST_ACCOUNT/ACCOUNT_COUNTRY/cd875303-942d-4cfe-86bf-3cfa7dc7309f.dict", + "TEST_KYLIN_FACT.SELLER_ID" : "/default/dict/DEFAULT.TEST_KYLIN_FACT/SELLER_ID/ee022220-805a-4be3-9289-d2f3efedcf83.dict", + "TEST_CATEGORY_GROUPINGS.META_CATEG_NAME" : "/default/dict/DEFAULT.TEST_CATEGORY_GROUPINGS/META_CATEG_NAME/f0bd22bc-98c4-4a7f-9b81-326d799a767d.dict", + "TEST_SITES.SITE_NAME" : "/default/dict/EDW.TEST_SITES/SITE_NAME/2fbb8eda-f0fb-4ab2-accb-79d0df1edb3f.dict", + "TEST_KYLIN_FACT.ORDER_ID" : "/default/dict/DEFAULT.TEST_KYLIN_FACT/ORDER_ID/2e2cd5e0-acc1-40cc-a719-8d242cecdd4f.dict", + "TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME" : "/default/dict/DEFAULT.TEST_CATEGORY_GROUPINGS/CATEG_LVL3_NAME/f853d029-7a0b-41e2-bf0e-8bb406771f78.dict" + }, + "snapshots" : { + "EDW.TEST_SITES" : "default/table_snapshot/EDW.TEST_SITES/c1e8096e-4e7f-4387-b7c3-5147c1ce38d6", + "TEST.COUNTRY" : "default/table_snapshot/DEFAULT.TEST_COUNTRY/b4849638-4eb5-44f5-b776-0619813fb676", + "EDW.TEST_SELLER_TYPE_DIM" : "default/table_snapshot/EDW.TEST_SELLER_TYPE_DIM/6fb67ffa-1143-4661-ac5d-a0633e308197", + "EDW.CAL_DT" : "default/table_snapshot/EDW.TEST_CAL_DT/395c529c-4e67-45d4-bd28-6e2712f7c8d5", + "TEST.TEST_CATEGORY_GROUPINGS" : "default/table_snapshot/DEFAULT.TEST_CATEGORY_GROUPINGS/135098c2-108f-41e9-a97c-e10fe75eac9b", + "TEST.TEST_ACCOUNT" : "default/table_snapshot/DEFAULT.TEST_ACCOUNT/d6ba492b-13bf-444d-b6e3-71bfa903344d", + "DEFAULT.TEST_ORDER" : "default/table_snapshot/DEFAULT.TEST_ORDER/3ad225e3-bec8-44c0-afab-ef8116e5385f" + }, + "last_build_time" : 0, + "source_count" : 10000, + "additionalInfo" : { } + } ], + "storage_location_identifier" : null +} \ No newline at end of file diff --git a/src/common-service/src/test/resources/update/default/model_desc/82fa7671-a935-45f5-8779-85703601f49a.json b/src/common-service/src/test/resources/update/default/model_desc/82fa7671-a935-45f5-8779-85703601f49a.json new file mode 100644 index 00000000000..adcfc04ec60 --- /dev/null +++ b/src/common-service/src/test/resources/update/default/model_desc/82fa7671-a935-45f5-8779-85703601f49a.json @@ -0,0 +1,1324 @@ +{ + "uuid": "82fa7671-a935-45f5-8779-85703601f49a", + "alias": "ut_inner_join_cube_partial", + "fact_table": "TEST.KYLIN_FACT", + "fact_table_alias": "TEST_KYLIN_FACT", + "owner": "ADMIN", + "join_tables": [ + { + "table": "DEFAULT.TEST_ORDER", + "alias": "TEST_ORDER", + "join": { + "type": "INNER", + "primary_key": [ + "TEST_ORDER.ORDER_ID" + ], + "foreign_key": [ + "TEST_KYLIN_FACT.ORDER_ID" + ] + } + }, + { + "table": "EDW.TEST_SELLER_TYPE_DIM", + "alias": "TEST_SELLER_TYPE_DIM", + "join": { + "type": "INNER", + "primary_key": [ + "TEST_SELLER_TYPE_DIM.SELLER_TYPE_CD" + ], + "foreign_key": [ + "TEST_KYLIN_FACT.SLR_SEGMENT_CD" + ] + } + }, + { + "table": "EDW.CAL_DT", + "alias": "TEST_CAL_DT", + "join": { + "type": "INNER", + "primary_key": [ + "TEST_CAL_DT.CAL_DT" + ], + "foreign_key": [ + "TEST_KYLIN_FACT.CAL_DT" + ] + } + }, + { + "table": "TEST.TEST_CATEGORY_GROUPINGS", + "alias": "TEST_CATEGORY_GROUPINGS", + "join": { + "type": "INNER", + "primary_key": [ + "TEST_CATEGORY_GROUPINGS.LEAF_CATEG_ID", + "TEST_CATEGORY_GROUPINGS.SITE_ID" + ], + "foreign_key": [ + "TEST_KYLIN_FACT.LEAF_CATEG_ID", + "TEST_KYLIN_FACT.LSTG_SITE_ID" + ] + } + }, + { + "table": "EDW.TEST_SITES", + "alias": "TEST_SITES", + "join": { + "type": "INNER", + "primary_key": [ + "TEST_SITES.SITE_ID" + ], + "foreign_key": [ + "TEST_KYLIN_FACT.LSTG_SITE_ID" + ] + } + }, + { + "table": "TEST.COUNTRY", + "alias": "BUYER_COUNTRY", + "join": { + "type": "INNER", + "primary_key": [ + "BUYER_COUNTRY.COUNTRY" + ], + "foreign_key": [ + "BUYER_ACCOUNT.ACCOUNT_COUNTRY" + ] + } + }, + { + "table": "TEST.COUNTRY", + "alias": "SELLER_COUNTRY", + "join": { + "type": "INNER", + "primary_key": [ + "SELLER_COUNTRY.COUNTRY" + ], + "foreign_key": [ + "SELLER_ACCOUNT.ACCOUNT_COUNTRY" + ] + } + }, + { + "table": "TEST.TEST_ACCOUNT", + "alias": "SELLER_ACCOUNT", + "join": { + "type": "INNER", + "primary_key": [ + "SELLER_ACCOUNT.ACCOUNT_ID" + ], + "foreign_key": [ + "TEST_KYLIN_FACT.SELLER_ID" + ] + } + }, + { + "table": "TEST.TEST_ACCOUNT", + "alias": "BUYER_ACCOUNT", + "join": { + "type": "INNER", + "primary_key": [ + "BUYER_ACCOUNT.ACCOUNT_ID" + ], + "foreign_key": [ + "TEST_ORDER.BUYER_ID" + ] + } + } + ], + "all_named_columns": [ + { + "id": 0, + "name": "SITE_NAME", + "column": "TEST_SITES.SITE_NAME", + "status": "DIMENSION" + }, + { + "id": 1, + "name": "TRANS_ID", + "column": "TEST_KYLIN_FACT.TRANS_ID", + "status": "DIMENSION" + }, + { + "id": 2, + "name": "CAL_DT", + "column": "TEST_KYLIN_FACT.CAL_DT", + "status": "DIMENSION" + }, + { + "id": 3, + "name": "LSTG_FORMAT_NAME", + "column": "TEST_KYLIN_FACT.LSTG_FORMAT_NAME", + "status": "DIMENSION" + }, + { + "id": 4, + "name": "LSTG_SITE_ID", + "column": "TEST_KYLIN_FACT.LSTG_SITE_ID", + "status": "DIMENSION" + }, + { + "id": 5, + "name": "META_CATEG_NAME", + "column": "TEST_CATEGORY_GROUPINGS.META_CATEG_NAME", + "status": "DIMENSION" + }, + { + "id": 6, + "name": "CATEG_LVL2_NAME", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME", + "status": "DIMENSION" + }, + { + "id": 7, + "name": "CATEG_LVL3_NAME", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME", + "status": "DIMENSION" + }, + { + "id": 8, + "name": "LEAF_CATEG_ID", + "column": "TEST_KYLIN_FACT.LEAF_CATEG_ID", + "status": "DIMENSION" + }, + { + "id": 9, + "name": "SELLER_ID", + "column": "TEST_KYLIN_FACT.SELLER_ID", + "status": "DIMENSION" + }, + { + "id": 10, + "name": "WEEK_BEG_DT", + "column": "TEST_CAL_DT.WEEK_BEG_DT", + "status": "TOMB" + }, + { + "id": 11, + "name": "PRICE", + "column": "TEST_KYLIN_FACT.PRICE" + }, + { + "id": 12, + "name": "ITEM_COUNT", + "column": "TEST_KYLIN_FACT.ITEM_COUNT" + }, + { + "id": 13, + "name": "ORDER_ID", + "column": "TEST_KYLIN_FACT.ORDER_ID", + "status": "DIMENSION" + }, + { + "id": 14, + "name": "TEST_DATE_ENC", + "column": "TEST_ORDER.TEST_DATE_ENC", + "status": "DIMENSION" + }, + { + "id": 15, + "name": "TEST_TIME_ENC", + "column": "TEST_ORDER.TEST_TIME_ENC", + "status": "DIMENSION" + }, + { + "id": 16, + "name": "SLR_SEGMENT_CD", + "column": "TEST_KYLIN_FACT.SLR_SEGMENT_CD", + "status": "DIMENSION" + }, + { + "id": 17, + "name": "BUYER_ID", + "column": "TEST_ORDER.BUYER_ID", + "status": "DIMENSION" + }, + { + "id": 18, + "name": "SELLER_BUYER_LEVEL", + "column": "SELLER_ACCOUNT.ACCOUNT_BUYER_LEVEL", + "status": "DIMENSION" + }, + { + "id": 19, + "name": "SELLER_SELLER_LEVEL", + "column": "SELLER_ACCOUNT.ACCOUNT_SELLER_LEVEL", + "status": "DIMENSION" + }, + { + "id": 20, + "name": "SELLER_COUNTRY", + "column": "SELLER_ACCOUNT.ACCOUNT_COUNTRY", + "status": "DIMENSION" + }, + { + "id": 21, + "name": "SELLER_COUNTRY_NAME", + "column": "SELLER_COUNTRY.NAME", + "status": "DIMENSION" + }, + { + "id": 22, + "name": "BUYER_BUYER_LEVEL", + "column": "BUYER_ACCOUNT.ACCOUNT_BUYER_LEVEL", + "status": "DIMENSION" + }, + { + "id": 23, + "name": "BUYER_SELLER_LEVEL", + "column": "BUYER_ACCOUNT.ACCOUNT_SELLER_LEVEL", + "status": "DIMENSION" + }, + { + "id": 24, + "name": "BUYER_COUNTRY", + "column": "BUYER_ACCOUNT.ACCOUNT_COUNTRY", + "status": "DIMENSION" + }, + { + "id": 25, + "name": "BUYER_COUNTRY_NAME", + "column": "BUYER_COUNTRY.NAME", + "status": "DIMENSION" + }, + { + "id": 26, + "name": "TEST_COUNT_DISTINCT_BITMAP", + "column": "TEST_KYLIN_FACT.TEST_COUNT_DISTINCT_BITMAP", + "status": "DIMENSION" + }, + { + "id": 27, + "name": "WEEK_END_DATE", + "column": "TEST_CAL_DT.WEEK_END_DATE" + }, + { + "id": 28, + "name": "AGE_FOR_RTL_QTR_ID", + "column": "TEST_CAL_DT.AGE_FOR_RTL_QTR_ID" + }, + { + "id": 29, + "name": "RTL_YEAR_BEG_DT", + "column": "TEST_CAL_DT.RTL_YEAR_BEG_DT" + }, + { + "id": 30, + "name": "DAY_OF_WEEK", + "column": "TEST_CAL_DT.DAY_OF_WEEK" + }, + { + "id": 31, + "name": "RTL_MONTH_BEG_DT", + "column": "TEST_CAL_DT.RTL_MONTH_BEG_DT" + }, + { + "id": 32, + "name": "DAY_OF_QTR_ID", + "column": "TEST_CAL_DT.DAY_OF_QTR_ID" + }, + { + "id": 33, + "name": "ORDER_ID", + "column": "TEST_ORDER.ORDER_ID" + }, + { + "id": 34, + "name": "SITES_UPD_DATE", + "column": "TEST_SITES.SITES_UPD_DATE" + }, + { + "id": 35, + "name": "CATEG_LVL4_NAME", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL4_NAME" + }, + { + "id": 36, + "name": "DAY_IN_QTR_COUNT", + "column": "TEST_CAL_DT.DAY_IN_QTR_COUNT" + }, + { + "id": 37, + "name": "CRE_DATE", + "column": "TEST_SITES.CRE_DATE" + }, + { + "id": 38, + "name": "QTR_END_DT", + "column": "TEST_CAL_DT.QTR_END_DT" + }, + { + "id": 39, + "name": "YEAR_ID", + "column": "TEST_CAL_DT.YEAR_ID" + }, + { + "id": 40, + "name": "AGE_FOR_RTL_WEEK_ID", + "column": "TEST_CAL_DT.AGE_FOR_RTL_WEEK_ID" + }, + { + "id": 41, + "name": "WEEK_BEG_DT", + "column": "TEST_CAL_DT.WEEK_BEG_DT" + }, + { + "id": 42, + "name": "LONGITUDE", + "column": "BUYER_COUNTRY.LONGITUDE" + }, + { + "id": 43, + "name": "CATEG_LVL6_ID", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL6_ID" + }, + { + "id": 44, + "name": "PRD_IND", + "column": "TEST_CAL_DT.PRD_IND" + }, + { + "id": 45, + "name": "IS_EFFECTUAL", + "column": "TEST_KYLIN_FACT.IS_EFFECTUAL" + }, + { + "id": 46, + "name": "YEAR_OF_CAL_ID", + "column": "TEST_CAL_DT.YEAR_OF_CAL_ID" + }, + { + "id": 47, + "name": "SAP_CATEGORY_ID", + "column": "TEST_CATEGORY_GROUPINGS.SAP_CATEGORY_ID" + }, + { + "id": 48, + "name": "COUNTRY", + "column": "BUYER_COUNTRY.COUNTRY" + }, + { + "id": 49, + "name": "SITE_DOMAIN_CODE", + "column": "TEST_SITES.SITE_DOMAIN_CODE" + }, + { + "id": 50, + "name": "BSNS_VRTCL_NAME", + "column": "TEST_CATEGORY_GROUPINGS.BSNS_VRTCL_NAME" + }, + { + "id": 51, + "name": "CATEG_BUSN_UNIT", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_BUSN_UNIT" + }, + { + "id": 52, + "name": "AGE_FOR_RTL_MONTH_ID", + "column": "TEST_CAL_DT.AGE_FOR_RTL_MONTH_ID" + }, + { + "id": 53, + "name": "CAL_DATE", + "column": "TEST_CAL_DT.CAL_DATE" + }, + { + "id": 54, + "name": "CATEG_LVL3_ID", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL3_ID" + }, + { + "id": 55, + "name": "AGE_FOR_MONTH_ID", + "column": "TEST_CAL_DT.AGE_FOR_MONTH_ID" + }, + { + "id": 56, + "name": "CAL_DT_NAME", + "column": "TEST_CAL_DT.CAL_DT_NAME" + }, + { + "id": 57, + "name": "CAL_DT_MNS_1YEAR_DT", + "column": "TEST_CAL_DT.CAL_DT_MNS_1YEAR_DT" + }, + { + "id": 58, + "name": "MONTH_END_DT", + "column": "TEST_CAL_DT.MONTH_END_DT" + }, + { + "id": 59, + "name": "SELLER_GROUP_DESC", + "column": "TEST_SELLER_TYPE_DIM.SELLER_GROUP_DESC" + }, + { + "id": 60, + "name": "CAL_DT_DESC", + "column": "TEST_CAL_DT.CAL_DT_DESC" + }, + { + "id": 61, + "name": "EOA_EMAIL_CSTMZBL_SITE_YN_ID", + "column": "TEST_SITES.EOA_EMAIL_CSTMZBL_SITE_YN_ID" + }, + { + "id": 62, + "name": "AGE_FOR_DT_ID", + "column": "TEST_CAL_DT.AGE_FOR_DT_ID" + }, + { + "id": 63, + "name": "ACCOUNT_ID", + "column": "BUYER_ACCOUNT.ACCOUNT_ID" + }, + { + "id": 64, + "name": "WTD_YN_ID", + "column": "TEST_CAL_DT.WTD_YN_ID" + }, + { + "id": 65, + "name": "MONTH_OF_CAL_ID", + "column": "TEST_CAL_DT.MONTH_OF_CAL_ID" + }, + { + "id": 66, + "name": "RTL_QTR_OF_RTL_YEAR_ID", + "column": "TEST_CAL_DT.RTL_QTR_OF_RTL_YEAR_ID" + }, + { + "id": 67, + "name": "AGE_FOR_QTR_ID", + "column": "TEST_CAL_DT.AGE_FOR_QTR_ID" + }, + { + "id": 68, + "name": "DAY_OF_YEAR_ID", + "column": "TEST_CAL_DT.DAY_OF_YEAR_ID" + }, + { + "id": 69, + "name": "YEAR_BEG_DT", + "column": "TEST_CAL_DT.YEAR_BEG_DT" + }, + { + "id": 70, + "name": "SELLER_GROUP_CD", + "column": "TEST_SELLER_TYPE_DIM.SELLER_GROUP_CD" + }, + { + "id": 71, + "name": "YEAR_IND", + "column": "TEST_CAL_DT.YEAR_IND" + }, + { + "id": 72, + "name": "LATITUDE", + "column": "SELLER_COUNTRY.LATITUDE" + }, + { + "id": 73, + "name": "ACCOUNT_CONTACT", + "column": "BUYER_ACCOUNT.ACCOUNT_CONTACT" + }, + { + "id": 74, + "name": "LONGITUDE", + "column": "SELLER_COUNTRY.LONGITUDE" + }, + { + "id": 75, + "name": "WEEK_IND", + "column": "TEST_CAL_DT.WEEK_IND" + }, + { + "id": 76, + "name": "CURR_CAL_DT_MNS_2QTR_YN_ID", + "column": "TEST_CAL_DT.CURR_CAL_DT_MNS_2QTR_YN_ID" + }, + { + "id": 77, + "name": "RETAIL_WEEK", + "column": "TEST_CAL_DT.RETAIL_WEEK" + }, + { + "id": 78, + "name": "QTR_BEG_DT", + "column": "TEST_CAL_DT.QTR_BEG_DT" + }, + { + "id": 79, + "name": "CAL_DT_CRE_DATE", + "column": "TEST_CAL_DT.CAL_DT_CRE_DATE" + }, + { + "id": 80, + "name": "CAL_DT_MNS_2QTR_DT", + "column": "TEST_CAL_DT.CAL_DT_MNS_2QTR_DT" + }, + { + "id": 81, + "name": "MOVE_TO", + "column": "TEST_CATEGORY_GROUPINGS.MOVE_TO" + }, + { + "id": 82, + "name": "CAL_DT_UPD_USER", + "column": "TEST_CAL_DT.CAL_DT_UPD_USER" + }, + { + "id": 83, + "name": "SITE_CNTRY_ID", + "column": "TEST_SITES.SITE_CNTRY_ID" + }, + { + "id": 84, + "name": "USER_DEFINED_FIELD1", + "column": "TEST_CATEGORY_GROUPINGS.USER_DEFINED_FIELD1" + }, + { + "id": 85, + "name": "CAL_DT_SHORT_NAME", + "column": "TEST_CAL_DT.CAL_DT_SHORT_NAME" + }, + { + "id": 86, + "name": "YTM_YN_ID", + "column": "TEST_CAL_DT.YTM_YN_ID" + }, + { + "id": 87, + "name": "PRD_DESC", + "column": "TEST_CAL_DT.PRD_DESC" + }, + { + "id": 88, + "name": "CAL_DT_MNS_1MONTH_DT", + "column": "TEST_CAL_DT.CAL_DT_MNS_1MONTH_DT" + }, + { + "id": 89, + "name": "CAL_DT_MNS_1QTR_DT", + "column": "TEST_CAL_DT.CAL_DT_MNS_1QTR_DT" + }, + { + "id": 90, + "name": "DFAULT_LSTG_CURNCY", + "column": "TEST_SITES.DFAULT_LSTG_CURNCY" + }, + { + "id": 91, + "name": "CATEG_FLAGS", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_FLAGS" + }, + { + "id": 92, + "name": "CATEG_LVL5_ID", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL5_ID" + }, + { + "id": 93, + "name": "GCS_ID", + "column": "TEST_CATEGORY_GROUPINGS.GCS_ID" + }, + { + "id": 94, + "name": "CATEG_LVL5_NAME", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL5_NAME" + }, + { + "id": 95, + "name": "GROUPINGS_CRE_DATE", + "column": "TEST_CATEGORY_GROUPINGS.GROUPINGS_CRE_DATE" + }, + { + "id": 96, + "name": "CAL_DT_UPD_DATE", + "column": "TEST_CAL_DT.CAL_DT_UPD_DATE" + }, + { + "id": 97, + "name": "YEAR_END_DT", + "column": "TEST_CAL_DT.YEAR_END_DT" + }, + { + "id": 98, + "name": "CRE_USER", + "column": "TEST_SITES.CRE_USER" + }, + { + "id": 99, + "name": "GLBL_RPRT_SLR_SGMNT_CD", + "column": "TEST_SELLER_TYPE_DIM.GLBL_RPRT_SLR_SGMNT_CD" + }, + { + "id": 100, + "name": "COUNTRY", + "column": "SELLER_COUNTRY.COUNTRY" + }, + { + "id": 101, + "name": "USER_DEFINED_FIELD3", + "column": "TEST_CATEGORY_GROUPINGS.USER_DEFINED_FIELD3" + }, + { + "id": 102, + "name": "SEASON_OF_YEAR_ID", + "column": "TEST_CAL_DT.SEASON_OF_YEAR_ID" + }, + { + "id": 103, + "name": "USER_DEFINED_FIELD5", + "column": "TEST_CATEGORY_GROUPINGS.USER_DEFINED_FIELD5" + }, + { + "id": 104, + "name": "GROUPINGS_CRE_USER", + "column": "TEST_CATEGORY_GROUPINGS.GROUPINGS_CRE_USER" + }, + { + "id": 105, + "name": "CURR_CAL_DT_MNS_1WEEK_YN_IND", + "column": "TEST_CAL_DT.CURR_CAL_DT_MNS_1WEEK_YN_IND" + }, + { + "id": 106, + "name": "SRC_ID", + "column": "TEST_CATEGORY_GROUPINGS.SRC_ID" + }, + { + "id": 107, + "name": "DIM_UPD_DATE", + "column": "TEST_SELLER_TYPE_DIM.DIM_UPD_DATE" + }, + { + "id": 108, + "name": "MONTH_BEG_DT", + "column": "TEST_CAL_DT.MONTH_BEG_DT" + }, + { + "id": 109, + "name": "CATEG_BUSN_MGR", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_BUSN_MGR" + }, + { + "id": 110, + "name": "YTW_YN_ID", + "column": "TEST_CAL_DT.YTW_YN_ID" + }, + { + "id": 111, + "name": "CATEG_LVL2_ID", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL2_ID" + }, + { + "id": 112, + "name": "WEEK_IN_YEAR_ID", + "column": "TEST_CAL_DT.WEEK_IN_YEAR_ID" + }, + { + "id": 113, + "name": "PRD_FLAG", + "column": "TEST_CAL_DT.PRD_FLAG" + }, + { + "id": 114, + "name": "CAL_DT_MNS_2WEEK_DT", + "column": "TEST_CAL_DT.CAL_DT_MNS_2WEEK_DT" + }, + { + "id": 115, + "name": "CAL_DT_MNS_2MONTH_DT", + "column": "TEST_CAL_DT.CAL_DT_MNS_2MONTH_DT" + }, + { + "id": 116, + "name": "QTR_ID", + "column": "TEST_CAL_DT.QTR_ID" + }, + { + "id": 117, + "name": "AGE_FOR_RTL_YEAR_ID", + "column": "TEST_CAL_DT.AGE_FOR_RTL_YEAR_ID" + }, + { + "id": 118, + "name": "WEEK_BEG_DATE", + "column": "TEST_CAL_DT.WEEK_BEG_DATE" + }, + { + "id": 119, + "name": "CAL_DT_MNS_1WEEK_DT", + "column": "TEST_CAL_DT.CAL_DT_MNS_1WEEK_DT" + }, + { + "id": 120, + "name": "RETAIL_WK_END_DATE", + "column": "TEST_CAL_DT.RETAIL_WK_END_DATE" + }, + { + "id": 121, + "name": "VCS_ID", + "column": "TEST_CATEGORY_GROUPINGS.VCS_ID" + }, + { + "id": 122, + "name": "AGE_FOR_CS_WEEK_ID", + "column": "TEST_CAL_DT.AGE_FOR_CS_WEEK_ID" + }, + { + "id": 123, + "name": "WEEK_NUM_DESC", + "column": "TEST_CAL_DT.WEEK_NUM_DESC" + }, + { + "id": 124, + "name": "REGN_CATEG", + "column": "TEST_CATEGORY_GROUPINGS.REGN_CATEG" + }, + { + "id": 125, + "name": "RETAIL_YEAR", + "column": "TEST_CAL_DT.RETAIL_YEAR" + }, + { + "id": 126, + "name": "SELLER_TYPE_DESC", + "column": "TEST_SELLER_TYPE_DIM.SELLER_TYPE_DESC" + }, + { + "id": 127, + "name": "RTL_QTR_BEG_DT", + "column": "TEST_CAL_DT.RTL_QTR_BEG_DT" + }, + { + "id": 128, + "name": "DOMAIN_ID", + "column": "TEST_CATEGORY_GROUPINGS.DOMAIN_ID" + }, + { + "id": 129, + "name": "SELLER_TYPE_CD", + "column": "TEST_SELLER_TYPE_DIM.SELLER_TYPE_CD" + }, + { + "id": 130, + "name": "ACCOUNT_CONTACT", + "column": "SELLER_ACCOUNT.ACCOUNT_CONTACT" + }, + { + "id": 131, + "name": "DIM_UPD_USER", + "column": "TEST_SELLER_TYPE_DIM.DIM_UPD_USER" + }, + { + "id": 132, + "name": "CURR_CAL_DT_MNS_2MONTH_YN_ID", + "column": "TEST_CAL_DT.CURR_CAL_DT_MNS_2MONTH_YN_ID" + }, + { + "id": 133, + "name": "AGE_FOR_YEAR_ID", + "column": "TEST_CAL_DT.AGE_FOR_YEAR_ID" + }, + { + "id": 134, + "name": "CURR_CAL_DT_MNS_1YEAR_YN_ID", + "column": "TEST_CAL_DT.CURR_CAL_DT_MNS_1YEAR_YN_ID" + }, + { + "id": 135, + "name": "LEAF_CATEG_NAME", + "column": "TEST_CATEGORY_GROUPINGS.LEAF_CATEG_NAME" + }, + { + "id": 136, + "name": "CATEG_LVL7_ID", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL7_ID" + }, + { + "id": 137, + "name": "META_CATEG_ID", + "column": "TEST_CATEGORY_GROUPINGS.META_CATEG_ID" + }, + { + "id": 138, + "name": "DAY_OF_CAL_ID", + "column": "TEST_CAL_DT.DAY_OF_CAL_ID" + }, + { + "id": 139, + "name": "TEST_EXTENDED_COLUMN", + "column": "TEST_ORDER.TEST_EXTENDED_COLUMN" + }, + { + "id": 140, + "name": "RTL_MONTH_OF_RTL_YEAR_ID", + "column": "TEST_CAL_DT.RTL_MONTH_OF_RTL_YEAR_ID" + }, + { + "id": 141, + "name": "PRD_ID", + "column": "TEST_CAL_DT.PRD_ID" + }, + { + "id": 142, + "name": "CAL_DT_CRE_USER", + "column": "TEST_CAL_DT.CAL_DT_CRE_USER" + }, + { + "id": 143, + "name": "WEEK_ID", + "column": "TEST_CAL_DT.WEEK_ID" + }, + { + "id": 144, + "name": "MONTH_ID", + "column": "TEST_CAL_DT.MONTH_ID" + }, + { + "id": 145, + "name": "DIM_CRE_DATE", + "column": "TEST_SELLER_TYPE_DIM.DIM_CRE_DATE" + }, + { + "id": 146, + "name": "CURR_CAL_DT_MNS_1MONTH_YN_ID", + "column": "TEST_CAL_DT.CURR_CAL_DT_MNS_1MONTH_YN_ID" + }, + { + "id": 147, + "name": "DAY_OF_WEEK_ID", + "column": "TEST_CAL_DT.DAY_OF_WEEK_ID" + }, + { + "id": 148, + "name": "SEASON_BEG_DT", + "column": "TEST_CAL_DT.SEASON_BEG_DT" + }, + { + "id": 149, + "name": "CATEG_LVL7_NAME", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL7_NAME" + }, + { + "id": 150, + "name": "UPD_USER", + "column": "TEST_CATEGORY_GROUPINGS.UPD_USER" + }, + { + "id": 151, + "name": "LATITUDE", + "column": "BUYER_COUNTRY.LATITUDE" + }, + { + "id": 152, + "name": "DAY_OF_MONTH_ID", + "column": "TEST_CAL_DT.DAY_OF_MONTH_ID" + }, + { + "id": 153, + "name": "CATEG_LVL4_ID", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL4_ID" + }, + { + "id": 154, + "name": "CURR_CAL_DT_MNS_2WEEK_YN_IND", + "column": "TEST_CAL_DT.CURR_CAL_DT_MNS_2WEEK_YN_IND" + }, + { + "id": 155, + "name": "YTQ_YN_ID", + "column": "TEST_CAL_DT.YTQ_YN_ID" + }, + { + "id": 156, + "name": "WEEK_OF_YEAR_ID", + "column": "TEST_CAL_DT.WEEK_OF_YEAR_ID" + }, + { + "id": 157, + "name": "DAY_IN_MONTH_COUNT", + "column": "TEST_CAL_DT.DAY_IN_MONTH_COUNT" + }, + { + "id": 158, + "name": "QTR_DESC", + "column": "TEST_CAL_DT.QTR_DESC" + }, + { + "id": 159, + "name": "YTD_YN_ID", + "column": "TEST_CAL_DT.YTD_YN_ID" + }, + { + "id": 160, + "name": "SITE_ID", + "column": "TEST_CATEGORY_GROUPINGS.SITE_ID" + }, + { + "id": 161, + "name": "MONTH_OF_YEAR_ID", + "column": "TEST_CAL_DT.MONTH_OF_YEAR_ID" + }, + { + "id": 162, + "name": "CS_WEEK_BEG_DT", + "column": "TEST_CAL_DT.CS_WEEK_BEG_DT" + }, + { + "id": 163, + "name": "MONTH_OF_QTR_ID", + "column": "TEST_CAL_DT.MONTH_OF_QTR_ID" + }, + { + "id": 164, + "name": "CATEG_LVL6_NAME", + "column": "TEST_CATEGORY_GROUPINGS.CATEG_LVL6_NAME" + }, + { + "id": 165, + "name": "QTR_OF_CAL_ID", + "column": "TEST_CAL_DT.QTR_OF_CAL_ID" + }, + { + "id": 166, + "name": "CURR_CAL_DT_MNS_2YEAR_YN_ID", + "column": "TEST_CAL_DT.CURR_CAL_DT_MNS_2YEAR_YN_ID" + }, + { + "id": 167, + "name": "ADULT_CATEG_YN", + "column": "TEST_CATEGORY_GROUPINGS.ADULT_CATEG_YN" + }, + { + "id": 168, + "name": "MTD_YN_ID", + "column": "TEST_CAL_DT.MTD_YN_ID" + }, + { + "id": 169, + "name": "RETAIL_START_DATE", + "column": "TEST_CAL_DT.RETAIL_START_DATE" + }, + { + "id": 170, + "name": "DAY_IN_WEEK_COUNT", + "column": "TEST_CAL_DT.DAY_IN_WEEK_COUNT" + }, + { + "id": 171, + "name": "LEAF_CATEG_ID", + "column": "TEST_CATEGORY_GROUPINGS.LEAF_CATEG_ID" + }, + { + "id": 172, + "name": "QTR_IND", + "column": "TEST_CAL_DT.QTR_IND" + }, + { + "id": 173, + "name": "CAL_DT", + "column": "TEST_CAL_DT.CAL_DT" + }, + { + "id": 174, + "name": "QTR_OF_YEAR_ID", + "column": "TEST_CAL_DT.QTR_OF_YEAR_ID" + }, + { + "id": 175, + "name": "UPD_DATE", + "column": "TEST_CATEGORY_GROUPINGS.UPD_DATE" + }, + { + "id": 176, + "name": "ACCOUNT_ID", + "column": "SELLER_ACCOUNT.ACCOUNT_ID" + }, + { + "id": 177, + "name": "CURR_CAL_DT_MNS_1QTR_YN_ID", + "column": "TEST_CAL_DT.CURR_CAL_DT_MNS_1QTR_YN_ID" + }, + { + "id": 178, + "name": "AGE_FOR_WEEK_ID", + "column": "TEST_CAL_DT.AGE_FOR_WEEK_ID" + }, + { + "id": 179, + "name": "DIM_CRE_USER", + "column": "TEST_SELLER_TYPE_DIM.DIM_CRE_USER" + }, + { + "id": 180, + "name": "CAL_DT_MNS_2YEAR_DT", + "column": "TEST_CAL_DT.CAL_DT_MNS_2YEAR_DT" + }, + { + "id": 181, + "name": "RTL_WEEK_OF_RTL_YEAR_ID", + "column": "TEST_CAL_DT.RTL_WEEK_OF_RTL_YEAR_ID" + }, + { + "id": 182, + "name": "SITE_ID", + "column": "TEST_SITES.SITE_ID" + }, + { + "id": 183, + "name": "DAY_IN_YEAR_COUNT", + "column": "TEST_CAL_DT.DAY_IN_YEAR_COUNT" + }, + { + "id": 184, + "name": "WEEK_BEG_END_DESC_MD", + "column": "TEST_CAL_DT.WEEK_BEG_END_DESC_MD" + }, + { + "id": 185, + "name": "QTD_YN_ID", + "column": "TEST_CAL_DT.QTD_YN_ID" + }, + { + "id": 186, + "name": "SITES_UPD_USER", + "column": "TEST_SITES.SITES_UPD_USER" + }, + { + "id": 187, + "name": "WEEK_OF_CAL_ID", + "column": "TEST_CAL_DT.WEEK_OF_CAL_ID" + }, + { + "id": 188, + "name": "WEEK_BEG_END_DESC_MDY", + "column": "TEST_CAL_DT.WEEK_BEG_END_DESC_MDY" + }, + { + "id": 189, + "name": "WEEK_END_DT", + "column": "TEST_CAL_DT.WEEK_END_DT" + }, + { + "id": 190, + "name": "RTL_WEEK_BEG_DT", + "column": "TEST_CAL_DT.RTL_WEEK_BEG_DT" + } + ], + "all_measures": [ + { + "id": 100000, + "name": "TRANS_CNT", + "function": { + "expression": "COUNT", + "parameters": [ + { + "type": "constant", + "value": "1" + } + ], + "returntype": "bigint" + } + }, + { + "id": 100001, + "name": "GMV_SUM", + "function": { + "expression": "SUM", + "parameters": [ + { + "type": "column", + "value": "TEST_KYLIN_FACT.PRICE" + } + ], + "returntype": "decimal(19,4)" + } + }, + { + "id": 100002, + "name": "GMV_MIN", + "function": { + "expression": "MIN", + "parameters": [ + { + "type": "column", + "value": "TEST_KYLIN_FACT.PRICE" + } + ], + "returntype": "decimal(19,4)" + } + }, + { + "id": 100003, + "name": "GMV_MAX", + "function": { + "expression": "MAX", + "parameters": [ + { + "type": "column", + "value": "TEST_KYLIN_FACT.PRICE" + } + ], + "returntype": "decimal(19,4)" + } + }, + { + "id": 100004, + "name": "ITEM_COUNT_SUM", + "function": { + "expression": "SUM", + "parameters": [ + { + "type": "column", + "value": "TEST_KYLIN_FACT.ITEM_COUNT" + } + ], + "returntype": "bigint" + } + }, + { + "id": 100005, + "name": "ITEM_COUNT_MAX", + "function": { + "expression": "MAX", + "parameters": [ + { + "type": "column", + "value": "TEST_KYLIN_FACT.ITEM_COUNT" + } + ], + "returntype": "bigint" + } + }, + { + "id": 100006, + "name": "ITEM_COUNT_MIN", + "tomb": true, + "function": { + "expression": "MIN", + "parameters": [ + { + "type": "column", + "value": "TEST_KYLIN_FACT.ITEM_COUNT" + } + ], + "returntype": "bigint" + } + }, + { + "id": 100007, + "name": "SELLER_HLL", + "function": { + "expression": "COUNT_DISTINCT", + "parameters": [ + { + "type": "column", + "value": "TEST_KYLIN_FACT.SELLER_ID" + } + ], + "returntype": "hllc(10)" + } + }, + { + "id": 100008, + "name": "COUNT_DISTINCT", + "function": { + "expression": "COUNT_DISTINCT", + "parameters": [ + { + "type": "column", + "value": "TEST_KYLIN_FACT.LSTG_FORMAT_NAME" + } + ], + "returntype": "hllc(10)" + } + }, + { + "id": 100009, + "name": "TOP_SELLER", + "function": { + "expression": "TOP_N", + "parameters": [ + { + "type": "column", + "value": "TEST_KYLIN_FACT.PRICE" + }, + { + "type": "column", + "value": "TEST_KYLIN_FACT.SELLER_ID" + } + ], + "returntype": "topn(10000, 4)", + "configuration": { + "topn.encoding.TEST_KYLIN_FACT.SELLER_ID": "int:4" + } + } + }, + { + "id": 100010, + "name": "TEST_COUNT_DISTINCT_BITMAP", + "function": { + "expression": "COUNT_DISTINCT", + "parameters": [ + { + "type": "column", + "value": "TEST_KYLIN_FACT.TEST_COUNT_DISTINCT_BITMAP" + } + ], + "returntype": "bitmap" + } + }, + { + "id": 100011, + "name": "GVM_PERCENTILE", + "function": { + "expression": "PERCENTILE", + "parameters": [ + { + "type": "column", + "value": "TEST_KYLIN_FACT.PRICE" + } + ], + "returntype": "percentile(10000)" + } + } + ], + "column_correlations": [ + { + "name": "CATEGORY_HIERARCHY", + "correlation_type": "hierarchy", + "columns": [ + "TEST_CATEGORY_GROUPINGS.META_CATEG_NAME", + "TEST_CATEGORY_GROUPINGS.CATEG_LVL2_NAME", + "TEST_CATEGORY_GROUPINGS.CATEG_LVL3_NAME" + ] + }, + { + "name": "DATE_HIERARCHY", + "correlation_type": "hierarchy", + "columns": [ + "TEST_CAL_DT.WEEK_BEG_DT", + "TEST_KYLIN_FACT.CAL_DT" + ] + }, + { + "name": "SITE_JOINT", + "correlation_type": "joint", + "columns": [ + "TEST_KYLIN_FACT.LSTG_SITE_ID", + "TEST_SITES.SITE_NAME" + ] + } + ], + "computed_columns": [], + "last_modified": 1422435345352, + "filter_condition": null, + "partition_desc": { + "partition_date_column": "TEST_KYLIN_FACT.CAL_DT", + "partition_date_start": 0, + "partition_date_format": "yyyy-MM-dd", + "partition_type": "APPEND" + } +} \ No newline at end of file diff --git a/src/common-service/src/test/resources/update/default/table/EDW.CAL_DT.json b/src/common-service/src/test/resources/update/default/table/EDW.CAL_DT.json new file mode 100644 index 00000000000..7adad88048a --- /dev/null +++ b/src/common-service/src/test/resources/update/default/table/EDW.CAL_DT.json @@ -0,0 +1,414 @@ +{ + + "uuid" : "0ff420eb-79ad-40bd-bca9-12d8cd05c60a", + "name" : "CAL_DT", + "columns" : [ { + "id" : "1", + "name" : "CAL_DT", + "datatype" : "date", + "index": "T" + }, { + "id" : "2", + "name" : "YEAR_BEG_DT", + "datatype" : "date", + "index": "T" + }, { + "id" : "3", + "name" : "QTR_BEG_DT", + "datatype" : "date", + "index": "T" + }, { + "id" : "4", + "name" : "MONTH_BEG_DT", + "datatype" : "date", + "index": "T" + }, { + "id" : "5", + "name" : "WEEK_BEG_DT", + "datatype" : "date", + "index": "T" + }, { + "id" : "6", + "name" : "AGE_FOR_YEAR_ID", + "datatype" : "smallint" + }, { + "id" : "7", + "name" : "AGE_FOR_QTR_ID", + "datatype" : "smallint" + }, { + "id" : "8", + "name" : "AGE_FOR_MONTH_ID", + "datatype" : "smallint" + }, { + "id" : "9", + "name" : "AGE_FOR_WEEK_ID", + "datatype" : "smallint" + }, { + "id" : "10", + "name" : "AGE_FOR_DT_ID", + "datatype" : "smallint" + }, { + "id" : "11", + "name" : "AGE_FOR_RTL_YEAR_ID", + "datatype" : "smallint" + }, { + "id" : "12", + "name" : "AGE_FOR_RTL_QTR_ID", + "datatype" : "smallint" + }, { + "id" : "13", + "name" : "AGE_FOR_RTL_MONTH_ID", + "datatype" : "smallint" + }, { + "id" : "14", + "name" : "AGE_FOR_RTL_WEEK_ID", + "datatype" : "smallint" + }, { + "id" : "15", + "name" : "AGE_FOR_CS_WEEK_ID", + "datatype" : "smallint" + }, { + "id" : "16", + "name" : "DAY_OF_CAL_ID", + "datatype" : "integer" + }, { + "id" : "17", + "name" : "DAY_OF_YEAR_ID", + "datatype" : "smallint" + }, { + "id" : "18", + "name" : "DAY_OF_QTR_ID", + "datatype" : "smallint" + }, { + "id" : "19", + "name" : "DAY_OF_MONTH_ID", + "datatype" : "smallint" + }, { + "id" : "20", + "name" : "DAY_OF_WEEK_ID", + "datatype" : "integer" + }, { + "id" : "21", + "name" : "WEEK_OF_YEAR_ID", + "datatype" : "tinyint" + }, { + "id" : "22", + "name" : "WEEK_OF_CAL_ID", + "datatype" : "integer" + }, { + "id" : "23", + "name" : "MONTH_OF_QTR_ID", + "datatype" : "tinyint" + }, { + "id" : "24", + "name" : "MONTH_OF_YEAR_ID", + "datatype" : "tinyint" + }, { + "id" : "25", + "name" : "MONTH_OF_CAL_ID", + "datatype" : "smallint" + }, { + "id" : "26", + "name" : "QTR_OF_YEAR_ID", + "datatype" : "tinyint" + }, { + "id" : "27", + "name" : "QTR_OF_CAL_ID", + "datatype" : "smallint" + }, { + "id" : "28", + "name" : "YEAR_OF_CAL_ID", + "datatype" : "smallint" + }, { + "id" : "29", + "name" : "YEAR_END_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "30", + "name" : "QTR_END_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "31", + "name" : "MONTH_END_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "32", + "name" : "WEEK_END_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "33", + "name" : "CAL_DT_NAME", + "datatype" : "varchar(4096)" + }, { + "id" : "34", + "name" : "CAL_DT_DESC", + "datatype" : "varchar(4096)" + }, { + "id" : "35", + "name" : "CAL_DT_SHORT_NAME", + "datatype" : "varchar(4096)" + }, { + "id" : "36", + "name" : "YTD_YN_ID", + "datatype" : "tinyint" + }, { + "id" : "37", + "name" : "QTD_YN_ID", + "datatype" : "tinyint" + }, { + "id" : "38", + "name" : "MTD_YN_ID", + "datatype" : "tinyint" + }, { + "id" : "39", + "name" : "WTD_YN_ID", + "datatype" : "tinyint" + }, { + "id" : "40", + "name" : "SEASON_BEG_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "41", + "name" : "DAY_IN_YEAR_COUNT", + "datatype" : "smallint" + }, { + "id" : "42", + "name" : "DAY_IN_QTR_COUNT", + "datatype" : "tinyint" + }, { + "id" : "43", + "name" : "DAY_IN_MONTH_COUNT", + "datatype" : "tinyint" + }, { + "id" : "44", + "name" : "DAY_IN_WEEK_COUNT", + "datatype" : "tinyint" + }, { + "id" : "45", + "name" : "RTL_YEAR_BEG_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "46", + "name" : "RTL_QTR_BEG_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "47", + "name" : "RTL_MONTH_BEG_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "48", + "name" : "RTL_WEEK_BEG_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "49", + "name" : "CS_WEEK_BEG_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "50", + "name" : "CAL_DATE", + "datatype" : "varchar(4096)" + }, { + "id" : "51", + "name" : "DAY_OF_WEEK", + "datatype" : "varchar(4096)" + }, { + "id" : "52", + "name" : "MONTH_ID", + "datatype" : "varchar(4096)" + }, { + "id" : "53", + "name" : "PRD_DESC", + "datatype" : "varchar(4096)" + }, { + "id" : "54", + "name" : "PRD_FLAG", + "datatype" : "varchar(4096)" + }, { + "id" : "55", + "name" : "PRD_ID", + "datatype" : "varchar(4096)" + }, { + "id" : "56", + "name" : "PRD_IND", + "datatype" : "varchar(4096)" + }, { + "id" : "57", + "name" : "QTR_DESC", + "datatype" : "varchar(4096)" + }, { + "id" : "58", + "name" : "QTR_ID", + "datatype" : "varchar(4096)" + }, { + "id" : "59", + "name" : "QTR_IND", + "datatype" : "varchar(4096)" + }, { + "id" : "60", + "name" : "RETAIL_WEEK", + "datatype" : "varchar(4096)" + }, { + "id" : "61", + "name" : "RETAIL_YEAR", + "datatype" : "varchar(4096)" + }, { + "id" : "62", + "name" : "RETAIL_START_DATE", + "datatype" : "varchar(4096)" + }, { + "id" : "63", + "name" : "RETAIL_WK_END_DATE", + "datatype" : "varchar(4096)" + }, { + "id" : "64", + "name" : "WEEK_IND", + "datatype" : "varchar(4096)" + }, { + "id" : "65", + "name" : "WEEK_NUM_DESC", + "datatype" : "varchar(4096)" + }, { + "id" : "66", + "name" : "WEEK_BEG_DATE", + "datatype" : "varchar(4096)" + }, { + "id" : "67", + "name" : "WEEK_END_DATE", + "datatype" : "varchar(4096)" + }, { + "id" : "68", + "name" : "WEEK_IN_YEAR_ID", + "datatype" : "varchar(4096)" + }, { + "id" : "69", + "name" : "WEEK_ID", + "datatype" : "varchar(4096)" + }, { + "id" : "70", + "name" : "WEEK_BEG_END_DESC_MDY", + "datatype" : "varchar(4096)" + }, { + "id" : "71", + "name" : "WEEK_BEG_END_DESC_MD", + "datatype" : "varchar(4096)" + }, { + "id" : "72", + "name" : "YEAR_ID", + "datatype" : "varchar(4096)" + }, { + "id" : "73", + "name" : "YEAR_IND", + "datatype" : "varchar(4096)" + }, { + "id" : "74", + "name" : "CAL_DT_MNS_1YEAR_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "75", + "name" : "CAL_DT_MNS_2YEAR_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "76", + "name" : "CAL_DT_MNS_1QTR_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "77", + "name" : "CAL_DT_MNS_2QTR_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "78", + "name" : "CAL_DT_MNS_1MONTH_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "79", + "name" : "CAL_DT_MNS_2MONTH_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "80", + "name" : "CAL_DT_MNS_1WEEK_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "81", + "name" : "CAL_DT_MNS_2WEEK_DT", + "datatype" : "varchar(4096)" + }, { + "id" : "82", + "name" : "CURR_CAL_DT_MNS_1YEAR_YN_ID", + "datatype" : "tinyint" + }, { + "id" : "83", + "name" : "CURR_CAL_DT_MNS_2YEAR_YN_ID", + "datatype" : "tinyint" + }, { + "id" : "84", + "name" : "CURR_CAL_DT_MNS_1QTR_YN_ID", + "datatype" : "tinyint" + }, { + "id" : "85", + "name" : "CURR_CAL_DT_MNS_2QTR_YN_ID", + "datatype" : "tinyint" + }, { + "id" : "86", + "name" : "CURR_CAL_DT_MNS_1MONTH_YN_ID", + "datatype" : "tinyint" + }, { + "id" : "87", + "name" : "CURR_CAL_DT_MNS_2MONTH_YN_ID", + "datatype" : "tinyint" + }, { + "id" : "88", + "name" : "CURR_CAL_DT_MNS_1WEEK_YN_IND", + "datatype" : "tinyint" + }, { + "id" : "89", + "name" : "CURR_CAL_DT_MNS_2WEEK_YN_IND", + "datatype" : "tinyint" + }, { + "id" : "90", + "name" : "RTL_MONTH_OF_RTL_YEAR_ID", + "datatype" : "varchar(4096)" + }, { + "id" : "91", + "name" : "RTL_QTR_OF_RTL_YEAR_ID", + "datatype" : "tinyint" + }, { + "id" : "92", + "name" : "RTL_WEEK_OF_RTL_YEAR_ID", + "datatype" : "tinyint" + }, { + "id" : "93", + "name" : "SEASON_OF_YEAR_ID", + "datatype" : "tinyint" + }, { + "id" : "94", + "name" : "YTM_YN_ID", + "datatype" : "tinyint" + }, { + "id" : "95", + "name" : "YTQ_YN_ID", + "datatype" : "tinyint" + }, { + "id" : "96", + "name" : "YTW_YN_ID", + "datatype" : "tinyint" + }, { + "id" : "97", + "name" : "CAL_DT_CRE_DATE", + "datatype" : "varchar(4096)" + }, { + "id" : "98", + "name" : "CAL_DT_CRE_USER", + "datatype" : "varchar(4096)" + }, { + "id" : "99", + "name" : "CAL_DT_UPD_DATE", + "datatype" : "varchar(4096)" + }, { + "id" : "100", + "name" : "CAL_DT_UPD_USER", + "datatype" : "varchar(4096)" + } ], + "database" : "edw", + "last_modified" : 0, + "source_type" : 9 +} \ No newline at end of file diff --git a/src/common-service/src/test/resources/update/default/table/TEST.COUNTRY.json b/src/common-service/src/test/resources/update/default/table/TEST.COUNTRY.json new file mode 100644 index 00000000000..34a926e9dea --- /dev/null +++ b/src/common-service/src/test/resources/update/default/table/TEST.COUNTRY.json @@ -0,0 +1,26 @@ +{ + "uuid" : "e286e39e-40d7-44c2-8fa2-41b365632882", + "name" : "COUNTRY", + "columns" : [ { + "id" : "1", + "name" : "COUNTRY", + "datatype" : "varchar(4096)", + "index": "T" + }, { + "id" : "2", + "name" : "LATITUDE", + "datatype" : "double" + }, { + "id" : "3", + "name" : "LONGITUDE", + "datatype" : "double" + }, { + "id" : "4", + "name" : "NAME", + "datatype" : "varchar(4096)", + "index": "T" + } ], + "database" : "TEST", + "last_modified" : 0, + "source_type" : 9 +} \ No newline at end of file diff --git a/src/common-service/src/test/resources/update/default/table/TEST.KYLIN_FACT.json b/src/common-service/src/test/resources/update/default/table/TEST.KYLIN_FACT.json new file mode 100644 index 00000000000..8bfff25729d --- /dev/null +++ b/src/common-service/src/test/resources/update/default/table/TEST.KYLIN_FACT.json @@ -0,0 +1,86 @@ +{ + "uuid" : "e286e39e-40d7-44c2-8fa2-41b365522771", + "name" : "KYLIN_FACT", + "data_gen" : "1", + "columns" : [ { + "id" : "1", + "name" : "TRANS_ID", + "datatype" : "bigint", + "data_gen" : "ID", + "comment": "TRANS_ID" + }, { + "id" : "2", + "name" : "ORDER_ID", + "datatype" : "bigint", + "index": "T", + "comment": "ORDER_ID" + }, { + "id" : "3", + "name" : "CAL_DT", + "datatype" : "date", + "data_gen" : "FK,order", + "index": "T", + "comment": "CAL_DT" + }, { + "id" : "4", + "name" : "LSTG_FORMAT_NAME", + "datatype" : "varchar(4096)", + "data_gen" : "FP-GTC|FP-non GTC|ABIN|Auction|Others", + "index": "T", + "comment": "LSTG_FORMAT_NAME" + }, { + "id" : "5", + "name" : "LEAF_CATEG_ID", + "datatype" : "bigint", + "data_gen" : "FK,null,nullstr=0", + "index": "T", + "comment": "LEAF_CATEG_ID" + }, { + "id" : "6", + "name" : "LSTG_SITE_ID", + "datatype" : "integer", + "index": "T", + "comment": "LSTG_SITE_ID" + }, { + "id" : "7", + "name" : "SLR_SEGMENT_CD", + "datatype" : "smallint", + "data_gen" : "FK,pk=EDW.TEST_SELLER_TYPE_DIM.SELLER_TYPE_CD", + "index": "T", + "comment": "SLR_SEGMENT_CD" + }, { + "id" : "8", + "name" : "SELLER_ID", + "datatype" : "integer", + "data_gen" : "RAND||10000000|10001000", + "index": "T", + "comment": "SELLER_ID" + }, { + "id" : "9", + "name" : "PRICE", + "datatype" : "decimal(19,4)", + "data_gen" : "RAND|.##|-100|1000", + "comment": "PRICE" + }, { + "id" : "10", + "name" : "ITEM_COUNT", + "datatype" : "integer", + "data_gen" : "RAND", + "comment": "ITEM_COUNT" + }, { + "id" : "11", + "name" : "TEST_COUNT_DISTINCT_BITMAP", + "datatype" : "varchar(4096)", + "data_gen" : "RAND", + "comment": "TEST_COUNT_DISTINCT_BITMAP" + }, { + "id" : "12", + "name" : "IS_EFFECTUAL", + "datatype" : "boolean", + "data_gen" : "true|false|TRUE|FALSE|True|False", + "comment": "IS_EFFECTUAL" + } ], + "database" : "TEST", + "last_modified" : 0, + "source_type" : 9 +} diff --git a/src/common-service/src/test/resources/update/default/table/TEST.TEST_ACCOUNT.json b/src/common-service/src/test/resources/update/default/table/TEST.TEST_ACCOUNT.json new file mode 100644 index 00000000000..43000fd9ca9 --- /dev/null +++ b/src/common-service/src/test/resources/update/default/table/TEST.TEST_ACCOUNT.json @@ -0,0 +1,41 @@ +{ + "uuid" : "f386e39e-40d7-44c2-9eb3-41b365632231", + "name" : "TEST_ACCOUNT", + "data_gen" : "2000", + + "columns" : [ { + "id" : "1", + "name" : "ACCOUNT_ID", + "datatype" : "bigint", + "data_gen" : "ID|10000000", + "index": "T", + "case_sensitive_name": "account_id" + }, { + "id" : "2", + "name" : "ACCOUNT_BUYER_LEVEL", + "datatype" : "integer", + "data_gen" : "RAND||0|5", + "case_sensitive_name": "account_buyer_level" + }, { + "id" : "3", + "name" : "ACCOUNT_SELLER_LEVEL", + "datatype" : "integer", + "data_gen" : "RAND||0|5", + "case_sensitive_name": "account_seller_level" + }, { + "id" : "4", + "name" : "ACCOUNT_COUNTRY", + "datatype" : "varchar(4096)", + "data_gen" : "CN|FR|GB|GE|JP|IT|RU|US", + "index": "T", + "case_sensitive_name": "account_country" + }, { + "id" : "5", + "name" : "ACCOUNT_CONTACT", + "datatype" : "varchar(4096)", + "case_sensitive_name": "account_contact" + } ], + "database" : "test", + "last_modified" : 0, + "source_type" : 9 +} \ No newline at end of file diff --git a/src/common-service/src/test/resources/update/default/table/TEST.TEST_CATEGORY_GROUPINGS.json b/src/common-service/src/test/resources/update/default/table/TEST.TEST_CATEGORY_GROUPINGS.json new file mode 100644 index 00000000000..37fc263b72c --- /dev/null +++ b/src/common-service/src/test/resources/update/default/table/TEST.TEST_CATEGORY_GROUPINGS.json @@ -0,0 +1,156 @@ +{ + + "uuid" : "952d11b5-69d9-45d1-92af-227489485e3f", + "name" : "TEST_CATEGORY_GROUPINGS", + "columns" : [ { + "id" : "1", + "name" : "LEAF_CATEG_ID", + "datatype" : "bigint", + "index": "T" + }, { + "id" : "2", + "name" : "LEAF_CATEG_NAME", + "datatype" : "varchar(4096)", + "index": "T" + }, { + "id" : "3", + "name" : "SITE_ID", + "datatype" : "integer", + "index": "T" + }, { + "id" : "4", + "name" : "CATEG_BUSN_MGR", + "datatype" : "varchar(4096)" + }, { + "id" : "5", + "name" : "CATEG_BUSN_UNIT", + "datatype" : "varchar(4096)" + }, { + "id" : "6", + "name" : "REGN_CATEG", + "datatype" : "varchar(4096)" + }, { + "id" : "7", + "name" : "USER_DEFINED_FIELD1", + "datatype" : "varchar(4096)" + }, { + "id" : "8", + "name" : "USER_DEFINED_FIELD3", + "datatype" : "varchar(4096)" + }, { + "id" : "9", + "name" : "GROUPINGS_CRE_DATE", + "datatype" : "varchar(4096)" + }, { + "id" : "10", + "name" : "UPD_DATE", + "datatype" : "varchar(4096)" + }, { + "id" : "11", + "name" : "GROUPINGS_CRE_USER", + "datatype" : "varchar(4096)" + }, { + "id" : "12", + "name" : "UPD_USER", + "datatype" : "varchar(4096)" + }, { + "id" : "13", + "name" : "META_CATEG_ID", + "datatype" : "decimal(19,4)" + }, { + "id" : "14", + "name" : "META_CATEG_NAME", + "datatype" : "varchar(4096)" + }, { + "id" : "15", + "name" : "CATEG_LVL2_ID", + "datatype" : "decimal(19,4)" + }, { + "id" : "16", + "name" : "CATEG_LVL3_ID", + "datatype" : "decimal(19,4)" + }, { + "id" : "17", + "name" : "CATEG_LVL4_ID", + "datatype" : "decimal(19,4)" + }, { + "id" : "18", + "name" : "CATEG_LVL5_ID", + "datatype" : "decimal(19,4)" + }, { + "id" : "19", + "name" : "CATEG_LVL6_ID", + "datatype" : "decimal(19,4)" + }, { + "id" : "20", + "name" : "CATEG_LVL7_ID", + "datatype" : "decimal(19,4)" + }, { + "id" : "21", + "name" : "CATEG_LVL2_NAME", + "datatype" : "varchar(4096)" + }, { + "id" : "22", + "name" : "CATEG_LVL3_NAME", + "datatype" : "varchar(4096)" + }, { + "id" : "23", + "name" : "CATEG_LVL4_NAME", + "datatype" : "varchar(4096)" + }, { + "id" : "24", + "name" : "CATEG_LVL5_NAME", + "datatype" : "varchar(4096)" + }, { + "id" : "25", + "name" : "CATEG_LVL6_NAME", + "datatype" : "varchar(4096)" + }, { + "id" : "26", + "name" : "CATEG_LVL7_NAME", + "datatype" : "varchar(4096)" + }, { + "id" : "27", + "name" : "CATEG_FLAGS", + "datatype" : "decimal(19,4)" + }, { + "id" : "28", + "name" : "ADULT_CATEG_YN", + "datatype" : "varchar(4096)" + }, { + "id" : "29", + "name" : "DOMAIN_ID", + "datatype" : "decimal(19,4)" + }, { + "id" : "30", + "name" : "USER_DEFINED_FIELD5", + "datatype" : "varchar(4096)" + }, { + "id" : "31", + "name" : "VCS_ID", + "datatype" : "decimal(19,4)" + }, { + "id" : "32", + "name" : "GCS_ID", + "datatype" : "decimal(19,4)" + }, { + "id" : "33", + "name" : "MOVE_TO", + "datatype" : "decimal(19,4)" + }, { + "id" : "34", + "name" : "SAP_CATEGORY_ID", + "datatype" : "decimal(19,4)" + }, { + "id" : "35", + "name" : "SRC_ID", + "datatype" : "tinyint" + }, { + "id" : "36", + "name" : "BSNS_VRTCL_NAME", + "datatype" : "varchar(4096)" + } ], + "database" : "test", + "last_modified" : 0, + "source_type" : 9 +} \ No newline at end of file diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataflow.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataflow.java index 7a5d81b145c..c3ca6f8e1ed 100644 --- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataflow.java +++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataflow.java @@ -25,6 +25,7 @@ import java.util.LinkedHashSet; import java.util.List; import java.util.Map; +import java.util.Objects; import java.util.Set; import java.util.stream.Collectors; @@ -603,4 +604,31 @@ public void initSegLayoutInfoById(Set segmentIdList) { getSegments(segmentIdList).forEach(NDataSegment::getLayoutInfo); } + + public boolean equalsRaw(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + NDataflow that = (NDataflow) o; + + if (!Objects.equals(uuid, that.uuid)) + return false; + if (!Objects.equals(status, that.status)) + return false; + if (!Objects.equals(lastStatus, that.lastStatus)) + return false; + if (!Objects.equals(cost, that.cost)) + return false; + if (!Objects.equals(queryHitCount, that.queryHitCount)) + return false; + if (!Objects.equals(lastQueryTime, that.lastQueryTime)) + return false; + return !getSegmentMaps(segments).equals(getSegmentMaps(that.segments)); + } + + private Map getSegmentMaps(Segments segments) { + return segments.stream().collect(Collectors.toMap(NDataSegment::getId, segment -> segment)); + } } diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataflowManager.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataflowManager.java index f22dd1838dc..52ca7d2eaf5 100644 --- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataflowManager.java +++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/cube/model/NDataflowManager.java @@ -48,6 +48,7 @@ import org.apache.kylin.common.exception.KylinException; import org.apache.kylin.common.persistence.ResourceStore; import org.apache.kylin.common.persistence.transaction.UnitOfWork; +import org.apache.kylin.common.persistence.Serializer; import org.apache.kylin.metadata.cachesync.CachedCrudAssist; import org.apache.kylin.metadata.model.ManagementType; import org.apache.kylin.metadata.model.NDataModel; @@ -582,6 +583,11 @@ public List getToRemoveSegs(NDataflow dataflow, NDataSegment segme return toRemoveSegs; } + // for test mostly + public Serializer getDataflowSerializer() { + return crud.getSerializer(); + } + public NDataflow copy(NDataflow df) { return crud.copyBySerialization(df); } diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java index b83b5f63186..0cbc31bd07f 100644 --- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java +++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/NDataModel.java @@ -1502,4 +1502,66 @@ public Set getEffectiveInternalMeasureIds() { return getEffectiveMeasures().values().stream().filter(m -> m.getType() == NDataModel.MeasureType.INTERNAL) .map(NDataModel.Measure::getId).collect(Collectors.toSet()); } + + public boolean equalsRaw(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + NDataModel that = (NDataModel) o; + + if (!Objects.equals(uuid, that.uuid)) + return false; + if (!Objects.equals(alias, that.alias)) + return false; + if (!Objects.equals(owner, that.owner)) + return false; + if (!Objects.equals(description, that.description)) + return false; + if (!Objects.equals(rootFactTableName, that.rootFactTableName)) + return false; + if (!Objects.equals(rootFactTableAlias, that.rootFactTableAlias)) + return false; + if (!getJoinTableMap(joinTables).equals(getJoinTableMap(that.joinTables))) + return false; + if (!Objects.equals(filterCondition, that.filterCondition)) + return false; + if (!Objects.equals(partitionDesc, that.partitionDesc)) + return false; + if (!Objects.equals(capacity, that.capacity)) + return false; + if (!Objects.equals(allNamedColumns, that.allNamedColumns)) + return false; + if (!Objects.equals(allMeasures, that.allMeasures)) + return false; + if (!Objects.equals(computedColumnDescs, that.computedColumnDescs)) + return false; + if (!Objects.equals(managementType, that.managementType)) + return false; + if (!Objects.equals(segmentConfig, that.segmentConfig)) + return false; + if (!Objects.equals(dataCheckDesc, that.dataCheckDesc)) + return false; + if (!Objects.equals(canvas, that.canvas)) + return false; + if (!Objects.equals(semanticVersion, that.semanticVersion)) + return false; + if (!Objects.equals(multiPartitionDesc, that.multiPartitionDesc)) + return false; + if (!Objects.equals(multiPartitionKeyMapping, that.multiPartitionKeyMapping)) + return false; + return Objects.equals(fusionId, that.fusionId); + } + + private Map getJoinTableMap(List joinTables) { + if (joinTables == null) { + return Maps.newHashMap(); + } + Map ret = Maps.newHashMapWithExpectedSize(joinTables.size()); + for (JoinTableDesc joinTable : joinTables) { + ret.put(joinTable.getAlias(), joinTable); + } + return ret; + } } diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/SegmentConfig.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/SegmentConfig.java index 9af96a8caef..cae1cd6e5be 100644 --- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/SegmentConfig.java +++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/SegmentConfig.java @@ -20,6 +20,7 @@ import java.io.Serializable; import java.util.List; +import java.util.Objects; import com.fasterxml.jackson.annotation.JsonProperty; @@ -57,4 +58,24 @@ public boolean canSkipHandleRetentionSegment() { return !retentionRange.isRetentionRangeEnabled() || retentionRange.getRetentionRangeNumber() <= 0 || retentionRange.getRetentionRangeType() == null; } + + @Override + public boolean equals(Object o) { + if (this == o) + return true; + if (o == null || getClass() != o.getClass()) + return false; + + SegmentConfig that = (SegmentConfig) o; + + if (!Objects.equals(autoMergeEnabled, that.autoMergeEnabled)) + return false; + if (!Objects.equals(autoMergeTimeRanges, that.autoMergeTimeRanges)) + return false; + if (!Objects.equals(volatileRange, that.volatileRange)) + return false; + if (!Objects.equals(retentionRange, that.retentionRange)) + return false; + return Objects.equals(createEmptySegmentEnabled, that.createEmptySegmentEnabled); + } } diff --git a/src/metadata-server/src/test/resources/update_hive_table/TableSchemaUpdateMapping.json b/src/metadata-server/src/test/resources/update_hive_table/TableSchemaUpdateMapping.json new file mode 100644 index 00000000000..bd601c81002 --- /dev/null +++ b/src/metadata-server/src/test/resources/update_hive_table/TableSchemaUpdateMapping.json @@ -0,0 +1,19 @@ +{ + "DEFAULT.TEST_KYLIN_FACT": { + "database": "TEST", + "tableName": "KYLIN_FACT" + }, + "DEFAULT.TEST_ACCOUNT": { + "database": "TEST" + }, + "DEFAULT.TEST_COUNTRY": { + "database": "TEST", + "tableName": "COUNTRY" + }, + "EDW.TEST_CAL_DT": { + "tableName": "CAL_DT" + }, + "DEFAULT.TEST_CATEGORY_GROUPINGS": { + "database": "TEST" + } +} \ No newline at end of file From 0eacd2b280171062f483c9707c0240e13d6f80fd Mon Sep 17 00:00:00 2001 From: zhennzhang Date: Sun, 8 Jan 2023 19:19:35 +0800 Subject: [PATCH 5/6] [EBAY-KYLIN-3788] support computed computed column mapping update. --- .../service/update/TableSchemaUpdater.java | 20 ++++++++++++++++--- .../apache/kylin/common/KylinConfigBase.java | 4 ++++ .../metadata/model/ComputedColumnDesc.java | 12 +++++++++++ 3 files changed, 33 insertions(+), 3 deletions(-) diff --git a/src/common-service/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdater.java b/src/common-service/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdater.java index 16791a6cafc..9bda182b19a 100644 --- a/src/common-service/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdater.java +++ b/src/common-service/src/main/java/org/apache/kylin/rest/service/update/TableSchemaUpdater.java @@ -27,6 +27,7 @@ import org.apache.kylin.metadata.cube.model.NDataSegment; import org.apache.kylin.metadata.cube.model.NDataflow; import org.apache.kylin.metadata.cube.model.NDataflowManager; +import org.apache.kylin.metadata.model.ComputedColumnDesc; import org.apache.kylin.metadata.model.JoinTableDesc; import org.apache.kylin.metadata.model.NDataModel; import org.apache.kylin.metadata.model.NDataModelManager; @@ -62,10 +63,9 @@ public static NDataModel dealWithMappingForModel(KylinConfig config, String proj throw new UnsupportedOperationException("Cannot deal with filter condition " + other.getFilterCondition()); } - // Currently, model with computed columns is not supported - if (other.getComputedColumnDescs().size() != 0) { + if ((!config.isSupportUpdateComputedColumnMapping()) && (other.getComputedColumnDescs().size() != 0)) { throw new UnsupportedOperationException( - "Cannot deal with filter condition " + other.getComputedColumnDescs()); + "Do not support deal with computed column " + other.getComputedColumnDescs()); } NDataModel copy = NDataModelManager.getInstance(config, project).copyForWrite(other); @@ -91,6 +91,20 @@ public static NDataModel dealWithMappingForModel(KylinConfig config, String proj } copy.setJoinTables(joinTablesCopy); + //mapping for computed columns + List computedColumns = other.getComputedColumnDescs(); + List computedColumnsCopy = new ArrayList<>(computedColumns.size()); + for (int i = 0; i < computedColumns.size(); i++) { + ComputedColumnDesc columnDesc = computedColumns.get(i); + computedColumnsCopy.add(ComputedColumnDesc.getCopyOf(columnDesc)); + String tableIdentity = columnDesc.getTableIdentity(); + TableSchemaUpdateMapping mapping = getTableSchemaUpdateMapping(mappings, tableIdentity); + if (mapping != null && mapping.isTableIdentityChanged()) { + computedColumnsCopy.get(i).setTableIdentity(mapping.getTableIdentity(tableIdentity)); + } + } + copy.setComputedColumnDescs(computedColumnsCopy); + return copy; } diff --git a/src/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java b/src/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java index 64606df2791..fa36028afaf 100644 --- a/src/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java +++ b/src/core-common/src/main/java/org/apache/kylin/common/KylinConfigBase.java @@ -740,6 +740,10 @@ public boolean isSemiAutoMode() { return Boolean.parseBoolean(getOptional("kylin.metadata.semi-automatic-mode", FALSE)); } + public boolean isSupportUpdateComputedColumnMapping() { + return Boolean.parseBoolean(getOptional("kylin.metadata.support-update-computed-column-mapping", FALSE)); + } + /** * expose computed column in the table metadata and select * queries * diff --git a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ComputedColumnDesc.java b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ComputedColumnDesc.java index 5cceedd5df4..adfeb2cf80c 100644 --- a/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ComputedColumnDesc.java +++ b/src/core-metadata/src/main/java/org/apache/kylin/metadata/model/ComputedColumnDesc.java @@ -200,6 +200,18 @@ public String getIdentName() { return tableIdentity + "." + columnName; } + public static ComputedColumnDesc getCopyOf(ComputedColumnDesc other) { + ComputedColumnDesc copy = new ComputedColumnDesc(); + copy.tableIdentity = other.tableIdentity; + copy.columnName = other.columnName; + copy.expression = other.expression; + copy.innerExpression = other.innerExpression; + copy.datatype = other.datatype; + copy.comment = other.comment; + copy.uuid = other.uuid; + return copy; + } + public void setInnerExpression(String innerExpression) { this.innerExpression = innerExpression; } From 759d29eb5b8d7d18ea640231fbe3b4b4107285da Mon Sep 17 00:00:00 2001 From: zhennzhang Date: Mon, 9 Jan 2023 13:48:53 +0800 Subject: [PATCH 6/6] [KYLIN-5388] fix code style. --- .../main/java/org/apache/kylin/rest/service/BasicService.java | 4 ++++ .../org/apache/kylin/rest/controller/NTableController.java | 2 ++ 2 files changed, 6 insertions(+) diff --git a/src/common-service/src/main/java/org/apache/kylin/rest/service/BasicService.java b/src/common-service/src/main/java/org/apache/kylin/rest/service/BasicService.java index adf0997aa8d..81c361b7687 100644 --- a/src/common-service/src/main/java/org/apache/kylin/rest/service/BasicService.java +++ b/src/common-service/src/main/java/org/apache/kylin/rest/service/BasicService.java @@ -137,4 +137,8 @@ public boolean remoteRequest(BroadcastEventReadyNotifier notifier, String projec } return true; } + + public NProjectManager getProjectManager(){ + return NProjectManager.getInstance(getConfig()); + } } diff --git a/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NTableController.java b/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NTableController.java index be17ad69fc4..54cb7c82d9b 100644 --- a/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NTableController.java +++ b/src/metadata-server/src/main/java/org/apache/kylin/rest/controller/NTableController.java @@ -34,6 +34,7 @@ import javax.servlet.http.HttpServletRequest; +import lombok.extern.slf4j.Slf4j; import org.apache.commons.collections.CollectionUtils; import org.apache.commons.lang.ArrayUtils; import org.apache.commons.lang.StringUtils; @@ -90,6 +91,7 @@ import io.swagger.annotations.ApiOperation; import lombok.val; +@Slf4j @Controller @RequestMapping(value = "/api/tables", produces = { HTTP_VND_APACHE_KYLIN_JSON }) public class NTableController extends NBasicController {