ResourceStore hbaseRS = ResourceStore.getStore(kylinConfHbase); ResourceStore hdfsRS = ResourceStore.getStore(kylinConfHdfs); dictMgrHdfs = DictionaryManager.getInstance(kylinConfHdfs); dictMgrHbase = DictionaryManager.getInstance(kylinConfHbase); logger.info(dictInfoResource); DictionaryInfo dictInfoHdfs = dictMgrHdfs.getDictionaryInfo(dictInfoResource); DictionaryInfo dicInfoHbase = dictMgrHbase.trySaveNewDict(dictInfoHdfs.getDictionaryObject(), dictInfoHdfs);
public DictionaryInfo buildDictionary(TblColRef col, IReadableTable inpTable) throws IOException { return buildDictionary(col, inpTable, null); }
for (CubeSegment segment : mergingSegments) { if (segment.getDictResPath(col) != null) { DictionaryInfo dictInfo = dictMgr.getDictionaryInfo(segment.getDictResPath(col)); if (dictInfo != null && !dictInfos.contains(dictInfo)) { dictInfos.add(dictInfo); DictionaryInfo mergedDictInfo = dictMgr.mergeDictionary(dictInfos); String tblCol = col.getTableAlias() + ":" + col.getName(); String dictInfoPath = mergedDictInfo == null ? "" : mergedDictInfo.getResourcePath();
public DictionaryInfo saveDictionary(TblColRef col, IReadableTable inpTable, Dictionary<String> dictionary) throws IOException { DictionaryInfo dictInfo = createDictionaryInfo(col, inpTable); String dupInfo = checkDupByInfo(dictInfo); if (dupInfo != null) { logger.info("Identical dictionary input " + dictInfo.getInput() + ", reuse existing dictionary at " + dupInfo); return getDictionaryInfo(dupInfo); } return trySaveNewDict(dictionary, dictInfo); }
initDictInfo(newDict, newDictInfo); DictionaryInfo largestDictInfo = findLargestDictInfo(newDictInfo); if (largestDictInfo != null) { largestDictInfo = getDictionaryInfo(largestDictInfo.getResourcePath()); Dictionary<String> largestDictObject = largestDictInfo.getDictionaryObject(); if (largestDictObject.contains(newDict)) { } else if (newDict.contains(largestDictObject)) { logger.info("dictionary content " + newDict + " is by far the largest, save it"); return saveNewDict(newDictInfo); } else { logger.info("merge dict and save..."); return mergeDictionary(Lists.newArrayList(newDictInfo, largestDictInfo)); return saveNewDict(newDictInfo); String dupDict = checkDupByContent(newDictInfo, newDict); if (dupDict != null) { logger.info("Identical dictionary content, reuse existing dictionary at " + dupDict); return getDictionaryInfo(dupDict); return saveNewDict(newDictInfo);
private String checkDupByContent(DictionaryInfo dictInfo, Dictionary<String> dict) throws IOException { ResourceStore store = getStore(); NavigableSet<String> existings = store.listResources(dictInfo.getResourceDir()); if (existings == null) return null; logger.info("{} existing dictionaries of the same column", existings.size()); if (existings.size() > 100) { logger.warn("Too many dictionaries under {}, dict count: {}", dictInfo.getResourceDir(), existings.size()); } for (String existing : existings) { DictionaryInfo existingInfo = getDictionaryInfo(existing); if (existingInfo != null) { if ((config.isDictResuable() && existingInfo.getDictionaryObject().contains(dict)) || dict.equals(existingInfo.getDictionaryObject())) { return existing; } } } return null; }
private DictionaryManager getDictionaryManager() { return DictionaryManager.getInstance(config); }
HashSet<TblColRef> colsNeedMeringDict = new HashSet<TblColRef>(); HashSet<TblColRef> colsNeedCopyDict = new HashSet<TblColRef>(); DictionaryManager dictMgr = DictionaryManager.getInstance(conf); for (TblColRef col : dim.getColumnRefs()) { if (newSeg.getCubeDesc().getRowkey().isUseDictionary(col)) { String dictTable = (String) dictMgr.decideSourceData(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null)[0]; if (cubeDesc.getFactTable().equalsIgnoreCase(dictTable)) { colsNeedMeringDict.add(col); for (CubeSegment segment : mergingSegments) { logger.info("Including fact table dictionary of segment : " + segment); DictionaryInfo dictInfo = dictMgr.getDictionaryInfo(segment.getDictResPath(col)); dictInfos.add(dictInfo);
public Dictionary<String> getDictionary(String resourcePath) throws IOException { DictionaryInfo dictInfo = getDictionaryInfo(resourcePath); return dictInfo == null ? null : dictInfo.getDictionaryObject(); }
public DictionaryInfo buildDictionary(DataModelDesc model, String dict, TblColRef col, String factColumnsPath) throws IOException { logger.info("building dictionary for " + col); Object[] tmp = decideSourceData(model, dict, col, factColumnsPath); String srcTable = (String) tmp[0]; String srcCol = (String) tmp[1]; int srcColIdx = (Integer) tmp[2]; ReadableTable inpTable = (ReadableTable) tmp[3]; DictionaryInfo dictInfo = new DictionaryInfo(srcTable, srcCol, srcColIdx, col.getDatatype(), inpTable.getSignature(), inpTable.getColumnDelimeter()); String dupDict = checkDupByInfo(dictInfo); if (dupDict != null) { logger.info("Identical dictionary input " + dictInfo.getInput() + ", reuse existing dictionary at " + dupDict); return getDictionaryInfo(dupDict); } Dictionary<?> dictionary = DictionaryGenerator.buildDictionary(dictInfo, inpTable); return trySaveNewDict(dictionary, dictInfo); }
DictionaryManager dictMgr = DictionaryManager.getInstance(kylinConfig); Dictionary<String> mergedDict = dictMgr.getDictionary(mergedSeg.getDictResPath(col)); continue; } else { sourceDict = dictMgr.getDictionary(mergingSeg.getDictResPath(col));
/** * For the new segment, we need to create new dimension dictionaries by merging underlying * dictionaries. (https://issues.apache.org/jira/browse/KYLIN-2457, https://issues.apache.org/jira/browse/KYLIN-2800) * @param cube * @param newSeg * @throws IOException */ private void makeDictForNewSegment(KylinConfig conf, CubeInstance cube, CubeSegment newSeg, List<CubeSegment> mergingSegments) throws IOException { DictionaryManager dictMgr = DictionaryManager.getInstance(conf); CubeDesc cubeDesc = cube.getDescriptor(); for (TblColRef col : cubeDesc.getAllColumnsNeedDictionaryBuilt()) { logger.info("Merging fact table dictionary on : " + col); List<DictionaryInfo> dictInfos = new ArrayList<DictionaryInfo>(); for (CubeSegment segment : mergingSegments) { logger.info("Including fact table dictionary of segment : " + segment); if (segment.getDictResPath(col) != null) { DictionaryInfo dictInfo = dictMgr.getDictionaryInfo(segment.getDictResPath(col)); if (dictInfo != null && !dictInfos.contains(dictInfo)) { dictInfos.add(dictInfo); } else { logger.warn("Failed to load DictionaryInfo from " + segment.getDictResPath(col)); } } } mergeDictionaries(dictMgr, newSeg, dictInfos, col); } }
public DictionaryInfo mergeDictionary(List<DictionaryInfo> dicts) throws IOException { DictionaryInfo firstDictInfo = null; int totalSize = 0; for (DictionaryInfo info : dicts) { // check if (firstDictInfo == null) { firstDictInfo = info; } else { if (!firstDictInfo.isDictOnSameColumn(info)) { throw new IllegalArgumentException("Merging dictionaries are not structurally equal(regardless of signature)."); } } totalSize += info.getInput().getSize(); } if (firstDictInfo == null) { throw new IllegalArgumentException("DictionaryManager.mergeDictionary input cannot be null"); } DictionaryInfo newDictInfo = new DictionaryInfo(firstDictInfo); TableSignature signature = newDictInfo.getInput(); signature.setSize(totalSize); signature.setLastModifiedTime(System.currentTimeMillis()); signature.setPath("merged_with_no_original_path"); String dupDict = checkDupByInfo(newDictInfo); if (dupDict != null) { logger.info("Identical dictionary input " + newDictInfo.getInput() + ", reuse existing dictionary at " + dupDict); return getDictionaryInfo(dupDict); } Dictionary<?> newDict = DictionaryGenerator.mergeDictionaries(newDictInfo, dicts); return trySaveNewDict(newDict, newDictInfo); }
@SuppressWarnings("unchecked") public static Map<TblColRef, Dictionary<String>> writeDictionary(CubeSegment cubeSegment, Map<TblColRef, Dictionary<String>> dictionaryMap, long startOffset, long endOffset) { Map<TblColRef, Dictionary<String>> realDictMap = Maps.newHashMap(); for (Map.Entry<TblColRef, Dictionary<String>> entry : dictionaryMap.entrySet()) { final TblColRef tblColRef = entry.getKey(); final Dictionary<String> dictionary = entry.getValue(); IReadableTable.TableSignature signature = new IReadableTable.TableSignature(); signature.setLastModifiedTime(System.currentTimeMillis()); signature.setPath(String.format(Locale.ROOT, "streaming_%s_%s", startOffset, endOffset)); signature.setSize(endOffset - startOffset); DictionaryInfo dictInfo = new DictionaryInfo(tblColRef.getColumnDesc(), tblColRef.getDatatype(), signature); logger.info("writing dictionary for TblColRef:" + tblColRef.toString()); DictionaryManager dictionaryManager = DictionaryManager.getInstance(cubeSegment.getCubeDesc().getConfig()); try { DictionaryInfo realDict = dictionaryManager.trySaveNewDict(dictionary, dictInfo); cubeSegment.putDictResPath(tblColRef, realDict.getResourcePath()); realDictMap.put(tblColRef, (Dictionary<String>) realDict.getDictionaryObject()); } catch (IOException e) { throw new RuntimeException("error save dictionary for column:" + tblColRef, e); } } return realDictMap; }
@Test public void testBuildSaveDictionary() throws IOException { KylinConfig config = KylinConfig.getInstanceFromEnv(); DictionaryManager dictMgr = DictionaryManager.getInstance(config); DataModelManager metaMgr = DataModelManager.getInstance(config); DataModelDesc model = metaMgr.getDataModelDesc("test_kylin_inner_join_model_desc"); DictionaryInfo nullInfo = dictMgr.buildDictionary(col, MockupReadableTable.newNonExistTable("/a/path")); assertEquals(null, nullInfo); DictionaryInfo info1 = dictMgr.buildDictionary(col, MockupReadableTable.newSingleColumnTable("/a/path", "1", "2", "3")); assertEquals(3, info1.getDictionaryObject().getSize()); DictionaryInfo info2 = dictMgr.buildDictionary(col, MockupReadableTable.newSingleColumnTable("/a/path", "1", "2", "3")); assertTrue(info1 == info2); DictionaryInfo info3 = dictMgr.buildDictionary(col, MockupReadableTable.newSingleColumnTable("/a/different/path", "1", "2", "3")); assertTrue(info1 == info3); DictionaryInfo info4 = dictMgr.saveDictionary(col, MockupReadableTable.newNonExistTable("/a/path"), dict); assertTrue(info1 == info4); DictionaryInfo info5 = dictMgr.saveDictionary(col, MockupReadableTable.newNonExistTable("/a/path"), dict2); assertTrue(info1 != info5);
private Boolean checkNeedMerging(TblColRef col) throws IOException { Boolean ret = dictsNeedMerging.get(col); if (ret != null) return ret; else { ret = cubeDesc.getRowkey().isUseDictionary(col); if (ret) { String dictTable = (String) DictionaryManager.getInstance(config).decideSourceData(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null)[0]; ret = cubeDesc.getFactTable().equalsIgnoreCase(dictTable); } dictsNeedMerging.put(col, ret); return ret; } }
public DictionaryInfo trySaveNewDict(Dictionary<?> newDict, DictionaryInfo newDictInfo) throws IOException { String dupDict = checkDupByContent(newDictInfo, newDict); if (dupDict != null) { logger.info("Identical dictionary content " + newDict + ", reuse existing dictionary at " + dupDict); return getDictionaryInfo(dupDict); } newDictInfo.setDictionaryObject(newDict); newDictInfo.setDictionaryClass(newDict.getClass().getName()); save(newDictInfo); dictCache.put(newDictInfo.getResourcePath(), newDictInfo); return newDictInfo; }
CubeManager.clearCache(); ProjectManager.clearCache(); DictionaryManager.clearCache(); dictionaryManager = DictionaryManager.getInstance(getTestConfig()); lfn = cube.getDescriptor().findColumnRef("DEFAULT.TEST_KYLIN_FACT", "LSTG_FORMAT_NAME"); lsi = cube.getDescriptor().findColumnRef("DEFAULT.TEST_KYLIN_FACT", "CAL_DT"); values.add(new byte[] { 98, 98, 98 }); Dictionary<?> dict = DictionaryGenerator.buildDictionaryFromValueList(newDictInfo, values); dictionaryManager.trySaveNewDict(dict, newDictInfo); ((TrieDictionary) dict).dump(System.out);
@Test @Ignore("hive not ready") public void basic() throws Exception { CubeDesc cubeDesc = CubeDescManager.getInstance(getTestConfig()).getCubeDesc("test_kylin_cube_without_slr_desc"); TblColRef col = cubeDesc.findColumnRef("DEFAULT.TEST_CATEGORY_GROUPINGS", "META_CATEG_NAME"); DictionaryInfo info1 = dictMgr.buildDictionary(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null); System.out.println(JsonUtil.writeValueAsIndentString(info1)); DictionaryInfo info2 = dictMgr.buildDictionary(cubeDesc.getModel(), cubeDesc.getRowkey().getDictionary(col), col, null); System.out.println(JsonUtil.writeValueAsIndentString(info2)); assertTrue(info1.getUuid() == info2.getUuid()); assertTrue(info1 == dictMgr.getDictionaryInfo(info1.getResourcePath())); assertTrue(info2 == dictMgr.getDictionaryInfo(info2.getResourcePath())); assertTrue(info1.getDictionaryObject() == info2.getDictionaryObject()); touchDictValues(info1); }
} else { Dictionary<String> newDict = DictionaryGenerator.mergeDictionaries(DataType.getType(newDictInfo.getDataType()), dicts); return trySaveNewDict(newDict, newDictInfo);