diff --git a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java index 878ce6b95b1..4012774fdfd 100644 --- a/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java +++ b/processing/src/main/java/org/apache/carbondata/processing/store/CarbonFactDataHandlerModel.java @@ -312,7 +312,7 @@ public static CarbonFactDataHandlerModel createCarbonFactDataHandlerModel( } carbonFactDataHandlerModel.dataMapWriterlistener = listener; carbonFactDataHandlerModel.writingCoresCount = configuration.getWritingCoresCount(); - setNumberOfCores(carbonFactDataHandlerModel); + carbonFactDataHandlerModel.initNumberOfCores(); carbonFactDataHandlerModel.setVarcharDimIdxInNoDict(varcharDimIdxInNoDict); return carbonFactDataHandlerModel; } @@ -400,7 +400,7 @@ public static CarbonFactDataHandlerModel getCarbonFactDataHandlerModel(CarbonLoa loadModel.getSegmentId()), segmentProperties); carbonFactDataHandlerModel.dataMapWriterlistener = listener; - setNumberOfCores(carbonFactDataHandlerModel); + carbonFactDataHandlerModel.initNumberOfCores(); carbonFactDataHandlerModel .setColumnLocalDictGenMap(CarbonUtil.getLocalDictionaryModel(carbonTable)); carbonFactDataHandlerModel.setVarcharDimIdxInNoDict(varcharDimIdxInNoDict); @@ -570,6 +570,7 @@ public boolean isCompactionFlow() { */ public void setCompactionFlow(boolean compactionFlow) { isCompactionFlow = compactionFlow; + initNumberOfCores(); } /** @@ -683,30 +684,30 @@ public void setColumnLocalDictGenMap( this.columnLocalDictGenMap = columnLocalDictGenMap; } - private static void setNumberOfCores(CarbonFactDataHandlerModel model) { + private void initNumberOfCores() { // in compaction flow the measure with decimal type will come as spark decimal. // need to convert it to byte array. - if (model.isCompactionFlow()) { + if (this.isCompactionFlow()) { try { - model.numberOfCores = Integer.parseInt(CarbonProperties.getInstance() + this.numberOfCores = Integer.parseInt(CarbonProperties.getInstance() .getProperty(CarbonCommonConstants.NUM_CORES_COMPACTING, CarbonCommonConstants.NUM_CORES_DEFAULT_VAL)); } catch (NumberFormatException exc) { LOGGER.error("Configured value for property " + CarbonCommonConstants.NUM_CORES_COMPACTING + "is wrong.Falling back to the default value " + CarbonCommonConstants.NUM_CORES_DEFAULT_VAL); - model.numberOfCores = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL); + this.numberOfCores = Integer.parseInt(CarbonCommonConstants.NUM_CORES_DEFAULT_VAL); } } else { - model.numberOfCores = CarbonProperties.getInstance().getNumberOfCores(); + this.numberOfCores = CarbonProperties.getInstance().getNumberOfCores(); } - if (model.sortScope != null && model.sortScope.equals(SortScopeOptions.SortScope.GLOBAL_SORT)) { - model.numberOfCores = 1; + if (this.sortScope != null && this.sortScope.equals(SortScopeOptions.SortScope.GLOBAL_SORT)) { + this.numberOfCores = 1; } // Overriding it to the task specified cores. - if (model.getWritingCoresCount() > 0) { - model.numberOfCores = model.getWritingCoresCount(); + if (this.getWritingCoresCount() > 0) { + this.numberOfCores = this.getWritingCoresCount(); } }