You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by zh...@apache.org on 2019/11/21 15:43:09 UTC
[carbondata] branch master updated: [HOTFIX] Remove unnecessary Map
object in CarbonTable #3442
This is an automated email from the ASF dual-hosted git repository.
zhangzc pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/carbondata.git
The following commit(s) were added to refs/heads/master by this push:
new 577dc47 [HOTFIX] Remove unnecessary Map object in CarbonTable #3442
577dc47 is described below
commit 577dc4771e428dd86f0959008b2801b7eb998ce8
Author: Jacky Li <ja...@qq.com>
AuthorDate: Wed Nov 13 00:45:42 2019 +0800
[HOTFIX] Remove unnecessary Map object in CarbonTable #3442
Remove unnecessary Map object in CarbonTable
This closes #3442
---
.../cache/dictionary/ManageDictionaryAndBTree.java | 3 +-
.../carbondata/core/datamap/DataMapFilter.java | 7 +-
.../carbondata/core/datastore/TableSpec.java | 6 +-
.../IncrementalColumnDictionaryGenerator.java | 6 +-
.../generator/TableDictionaryGenerator.java | 8 +-
.../carbondata/core/metadata/CarbonMetadata.java | 3 +-
.../core/metadata/schema/table/CarbonTable.java | 440 ++++++---------------
.../carbondata/core/scan/model/QueryModel.java | 4 +-
.../core/scan/model/QueryModelBuilder.java | 14 +-
.../carbondata/core/stream/StreamPruner.java | 4 +-
.../apache/carbondata/core/util/CarbonUtil.java | 3 +-
.../apache/carbondata/core/util/DataTypeUtil.java | 5 +-
.../core/metadata/CarbonMetadataTest.java | 10 +-
.../metadata/schema/table/CarbonTableTest.java | 6 +-
.../table/CarbonTableWithComplexTypesTest.java | 4 +-
.../org/apache/carbondata/mv/datamap/MVUtil.scala | 2 +-
.../hadoop/api/CarbonFileInputFormat.java | 2 +-
.../hadoop/api/CarbonTableInputFormat.java | 2 +-
.../hadoop/stream/StreamRecordReader.java | 10 +-
.../carbondata/hadoop/testutil/StoreCreator.java | 2 +-
.../carbondata/hive/MapredCarbonInputFormat.java | 2 +-
.../carbondata/presto/impl/CarbonTableReader.java | 2 +-
.../presto/util/CarbonDataStoreCreator.scala | 6 +-
.../cluster/sdv/generated/BucketingTestCase.scala | 4 +-
.../testsuite/binary/TestBinaryDataType.scala | 2 +-
.../complexType/TestCreateTableWithDouble.scala | 2 +-
.../dataload/TestNoInvertedIndexLoadAndQuery.scala | 12 +-
...ryWithColumnMetCacheAndCacheLevelProperty.scala | 2 +-
.../longstring/VarcharDataTypesBasicTestCase.scala | 4 +-
.../partition/TestDDLForPartitionTable.scala | 6 +-
...DDLForPartitionTableWithDefaultProperties.scala | 8 +-
.../org/apache/carbondata/spark/util/Util.java | 2 +-
.../spark/rdd/AlterTableLoadPartitionRDD.scala | 4 +-
.../spark/rdd/CarbonScanPartitionRDD.scala | 3 +-
.../spark/rdd/NewCarbonDataLoadRDD.scala | 12 +-
.../carbondata/spark/rdd/PartitionDropper.scala | 4 +-
.../carbondata/spark/rdd/PartitionSplitter.scala | 4 +-
.../carbondata/spark/rdd/StreamHandoffRDD.scala | 5 +-
.../carbondata/spark/util/CarbonScalaUtil.scala | 2 +-
.../apache/carbondata/spark/util/CommonUtil.scala | 2 +-
.../spark/util/GlobalDictionaryUtil.scala | 3 +-
.../carbondata/streaming/StreamSinkFactory.scala | 2 +-
.../command/carbonTableSchemaCommon.scala | 22 +-
.../streaming/CarbonAppendableStreamSink.scala | 4 +-
.../datamap/IndexDataMapRebuildRDD.scala | 4 +-
.../spark/rdd/CarbonDataRDDFactory.scala | 4 +-
.../spark/rdd/CarbonTableCompactor.scala | 5 +-
.../carbondata/spark/util/CarbonSparkUtil.scala | 12 +-
.../carbondata/stream/StreamJobManager.scala | 2 +-
.../apache/spark/sql/CarbonDictionaryDecoder.scala | 10 +-
.../command/management/CarbonLoadDataCommand.scala | 15 +-
.../mutation/CarbonProjectForUpdateCommand.scala | 2 +-
.../CarbonAlterTableDropPartitionCommand.scala | 2 +-
.../CarbonAlterTableSplitPartitionCommand.scala | 2 +-
.../CarbonShowCarbonPartitionsCommand.scala | 3 +-
.../command/preaaggregate/PreAggregateUtil.scala | 2 +-
.../schema/CarbonAlterTableAddColumnCommand.scala | 2 +-
...nAlterTableColRenameDataTypeChangeCommand.scala | 4 +-
.../schema/CarbonAlterTableDropColumnCommand.scala | 6 +-
.../command/timeseries/TimeSeriesUtil.scala | 2 +-
.../strategy/CarbonLateDecodeStrategy.scala | 4 +-
.../spark/sql/hive/CarbonPreAggregateRules.scala | 2 +-
.../org/apache/spark/sql/hive/CarbonRelation.scala | 13 +-
.../org/apache/spark/util/AlterTableUtil.scala | 10 +-
.../scala/org/apache/spark/util/DataMapUtil.scala | 2 +-
.../partition/TestAlterPartitionTable.scala | 32 +-
.../spark/util/DictionaryTestCaseUtil.scala | 2 +-
.../bucketing/TableBucketingTestCase.scala | 4 +-
.../AlterTableColumnRenameTestCase.scala | 28 +-
log | 0
.../processing/loading/DataLoadProcessBuilder.java | 10 +-
.../loading/model/CarbonLoadModelBuilder.java | 6 +-
.../processing/merger/CarbonCompactionUtil.java | 8 +-
.../merger/CompactionResultSortProcessor.java | 2 +-
.../processing/sort/sortdata/TableFieldStat.java | 3 +-
.../store/CarbonFactDataHandlerModel.java | 8 +-
.../processing/util/CarbonDataProcessorUtil.java | 21 +-
.../carbondata/sdk/file/CarbonWriterBuilder.java | 2 +-
.../streaming/CarbonStreamRecordWriter.java | 3 +-
79 files changed, 331 insertions(+), 566 deletions(-)
diff --git a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
index 88c3ddb..571e737 100644
--- a/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
+++ b/core/src/main/java/org/apache/carbondata/core/cache/dictionary/ManageDictionaryAndBTree.java
@@ -96,8 +96,7 @@ public class ManageDictionaryAndBTree {
*/
public static void clearBTreeAndDictionaryLRUCache(CarbonTable carbonTable) {
// clear dictionary cache from LRU cache
- List<CarbonDimension> dimensions =
- carbonTable.getDimensionByTableName(carbonTable.getTableName());
+ List<CarbonDimension> dimensions = carbonTable.getVisibleDimensions();
for (CarbonDimension dimension : dimensions) {
removeDictionaryColumnFromCache(carbonTable.getAbsoluteTableIdentifier(),
dimension.getColumnId());
diff --git a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapFilter.java b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapFilter.java
index 23805e2..1ca9697 100644
--- a/core/src/main/java/org/apache/carbondata/core/datamap/DataMapFilter.java
+++ b/core/src/main/java/org/apache/carbondata/core/datamap/DataMapFilter.java
@@ -207,9 +207,10 @@ public class DataMapFilter implements Serializable {
new ArrayList<CarbonDimension>());
} else {
processVO =
- new QueryModel.FilterProcessVO(table.getDimensionByTableName(table.getTableName()),
- table.getMeasureByTableName(table.getTableName()),
- table.getImplicitDimensionByTableName(table.getTableName()));
+ new QueryModel.FilterProcessVO(
+ table.getVisibleDimensions(),
+ table.getVisibleMeasures(),
+ table.getImplicitDimensions());
}
QueryModel.processFilterExpression(processVO, expression, isFilterDimensions, isFilterMeasures,
table);
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
index 101a426..e34dc7a 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/TableSpec.java
@@ -58,10 +58,8 @@ public class TableSpec {
public TableSpec(CarbonTable carbonTable) {
this.carbonTable = carbonTable;
- List<CarbonDimension> dimensions =
- carbonTable.getDimensionByTableName(carbonTable.getTableName());
- List<CarbonMeasure> measures =
- carbonTable.getMeasureByTableName(carbonTable.getTableName());
+ List<CarbonDimension> dimensions = carbonTable.getVisibleDimensions();
+ List<CarbonMeasure> measures = carbonTable.getVisibleMeasures();
// first calculate total number of columnar field considering column group and complex column
numSimpleDimensions = 0;
for (CarbonDimension dimension : dimensions) {
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
index 66607dc..a3600e7 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/IncrementalColumnDictionaryGenerator.java
@@ -36,7 +36,7 @@ import org.apache.carbondata.core.devapi.DictionaryGenerator;
import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
import org.apache.carbondata.core.metadata.ColumnIdentifier;
import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
import org.apache.carbondata.core.service.CarbonCommonFactory;
import org.apache.carbondata.core.service.DictionaryService;
import org.apache.carbondata.core.util.CarbonUtil;
@@ -69,11 +69,11 @@ public class IncrementalColumnDictionaryGenerator implements BiDictionary<Intege
private int maxValue;
- private CarbonDimension dimension;
+ private CarbonColumn dimension;
private CarbonTable carbonTable;
- public IncrementalColumnDictionaryGenerator(CarbonDimension dimension, int maxValue,
+ public IncrementalColumnDictionaryGenerator(CarbonColumn dimension, int maxValue,
CarbonTable carbonTable) {
this.carbonTable = carbonTable;
this.maxValue = maxValue;
diff --git a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
index af81f33..b212291 100644
--- a/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
+++ b/core/src/main/java/org/apache/carbondata/core/dictionary/generator/TableDictionaryGenerator.java
@@ -30,7 +30,7 @@ import org.apache.carbondata.core.devapi.DictionaryGenerationException;
import org.apache.carbondata.core.devapi.DictionaryGenerator;
import org.apache.carbondata.core.dictionary.generator.key.DictionaryMessage;
import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
-import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
import org.apache.carbondata.core.util.CarbonProperties;
import org.apache.log4j.Logger;
@@ -57,7 +57,7 @@ public class TableDictionaryGenerator
@Override
public Integer generateKey(DictionaryMessage value)
throws DictionaryGenerationException {
- CarbonDimension dimension = carbonTable.getPrimitiveDimensionByName(value.getColumnName());
+ CarbonColumn dimension = carbonTable.getPrimitiveDimensionByName(value.getColumnName());
if (null == dimension) {
throw new DictionaryGenerationException("Dictionary Generation Failed");
@@ -68,7 +68,7 @@ public class TableDictionaryGenerator
}
public Integer size(DictionaryMessage key) {
- CarbonDimension dimension = carbonTable.getPrimitiveDimensionByName(key.getColumnName());
+ CarbonColumn dimension = carbonTable.getPrimitiveDimensionByName(key.getColumnName());
if (null == dimension) {
return 0;
@@ -98,7 +98,7 @@ public class TableDictionaryGenerator
}
public void updateGenerator(DictionaryMessage key) {
- CarbonDimension dimension = carbonTable
+ CarbonColumn dimension = carbonTable
.getPrimitiveDimensionByName(key.getColumnName());
if (null != dimension && null == columnMap.get(dimension.getColumnId())) {
synchronized (columnMap) {
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/CarbonMetadata.java b/core/src/main/java/org/apache/carbondata/core/metadata/CarbonMetadata.java
index c2adfd0..88f5e8d 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/CarbonMetadata.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/CarbonMetadata.java
@@ -141,8 +141,7 @@ public final class CarbonMetadata {
*/
public CarbonDimension getCarbonDimensionBasedOnColIdentifier(CarbonTable carbonTable,
String columnIdentifier) {
- List<CarbonDimension> listOfCarbonDims =
- carbonTable.getDimensionByTableName(carbonTable.getTableName());
+ List<CarbonDimension> listOfCarbonDims = carbonTable.getVisibleDimensions();
for (CarbonDimension dimension : listOfCarbonDims) {
if (dimension.getColumnId().equalsIgnoreCase(columnIdentifier)) {
return dimension;
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index 2c13ec1..bb0b28d 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -24,7 +24,6 @@ import java.io.Serializable;
import java.util.ArrayList;
import java.util.Collections;
import java.util.Comparator;
-import java.util.HashMap;
import java.util.Iterator;
import java.util.LinkedList;
import java.util.List;
@@ -74,44 +73,30 @@ import org.apache.log4j.Logger;
public class CarbonTable implements Serializable, Writable {
private static final Logger LOGGER = LogServiceFactory.getLogService(CarbonTable.class.getName());
- /**
- * serialization id
- */
private static final long serialVersionUID = 8696507171227156445L;
- /**
- * the cached table info
- */
+
+ // The main object that contains all carbon table information, including
+ // schema, store path, table properties, datamap related info, etc.
+ // All other fields in CarbonTable can be derived from TableInfo.
private TableInfo tableInfo;
- /**
- * TableName, Dimensions list. This map will contain allDimensions which are visible
- */
- private Map<String, List<CarbonDimension>> tableDimensionsMap;
- /**
- * list of all the allDimensions
- */
+ // Visible dimension columns that exposed to user (can be queried)
+ private List<CarbonDimension> visibleDimensions;
+
+ // All dimension columns including visible columns and implicit columns
private List<CarbonDimension> allDimensions;
- private Map<String, List<CarbonColumn>> createOrderColumn;
+ // An ordered list, same order as when creating this table by user
+ private List<CarbonColumn> createOrderColumn;
- /**
- * TableName, Dimensions and children allDimensions list
- */
- private Map<String, List<CarbonDimension>> tablePrimitiveDimensionsMap;
+ // Implicit columns that for internal usage, like positionid and tupleid for update/delete
+ // operation. see CARBON_IMPLICIT_COLUMN_POSITIONID, CARBON_IMPLICIT_COLUMN_TUPLEID
+ private List<CarbonDimension> implicitDimensions;
- /**
- * table allMeasures list.
- */
- private Map<String, List<CarbonDimension>> tableImplicitDimensionsMap;
+ // Visible measure columns that exposed to user (can be queried)
+ private List<CarbonMeasure> visibleMeasures;
- /**
- * table allMeasures list. This map will contain allDimensions which are visible
- */
- private Map<String, List<CarbonMeasure>> tableMeasuresMap;
-
- /**
- * list of allMeasures
- */
+ // All measure columns including visible columns and implicit columns
private List<CarbonMeasure> allMeasures;
/**
@@ -119,76 +104,40 @@ public class CarbonTable implements Serializable, Writable {
*/
private List<CarbonDimension> columnDrift;
- /**
- * table bucket map.
- */
- private Map<String, BucketingInfo> tableBucketMap;
-
- /**
- * table partition info
- */
- private Map<String, PartitionInfo> tablePartitionMap;
+ // Bucket information defined by user when creating table
+ // Will be deleted after 2.0
+ @Deprecated
+ private BucketingInfo bucket;
- /**
- * tableUniqueName
- */
- private String tableUniqueName;
-
- /**
- * last updated time
- */
- private long tableLastUpdatedTime;
+ // Partition (Range/List/Hash). This is not for Hive partition.
+ // Will be deleted after 2.0
+ @Deprecated
+ private PartitionInfo partition;
- /**
- * table block size in MB
- */
- private int blockSize;
-
- /**
- * the number of columns in SORT_COLUMNS
- */
+ // Number of columns in SORT_COLUMNS table property
private int numberOfSortColumns;
- /**
- * the number of no dictionary columns in SORT_COLUMNS
- */
+ // Number of no dictionary columns in SORT_COLUMNS
private int numberOfNoDictSortColumns;
+ // The last index of the dimension column in all columns
private int dimensionOrdinalMax;
+ // True if this table has datamap
private boolean hasDataMapSchema;
- /**
- * is local dictionary generation enabled for the table
- */
+ // True if local dictionary is enabled for this table
private boolean isLocalDictionaryEnabled;
- /**
- * local dictionary generation threshold
- */
+ // Cardinality threshold for local dictionary, below which dictionary will be generated
private int localDictionaryThreshold;
- /**
- * The boolean field which points if the data written for Non Transactional Table
- * or Transactional Table.
- * transactional table means carbon will provide transactional support when user doing data
- * management like data loading, whether it is success or failure, data will be in consistent
- * state
- * The difference between Transactional and non Transactional table is
- * non Transactional Table will not contain any Metadata folder and subsequently
- * no TableStatus or Schema files.
- */
- private boolean isTransactionalTable = true;
-
public CarbonTable() {
- this.tableDimensionsMap = new HashMap<String, List<CarbonDimension>>();
- this.tableImplicitDimensionsMap = new HashMap<String, List<CarbonDimension>>();
- this.tableMeasuresMap = new HashMap<String, List<CarbonMeasure>>();
- this.tableBucketMap = new HashMap<>();
- this.tablePartitionMap = new HashMap<>();
- this.createOrderColumn = new HashMap<String, List<CarbonColumn>>();
- this.tablePrimitiveDimensionsMap = new HashMap<String, List<CarbonDimension>>();
- this.columnDrift = new ArrayList<CarbonDimension>();
+ this.visibleDimensions = new LinkedList<>();
+ this.implicitDimensions = new LinkedList<>();
+ this.visibleMeasures = new LinkedList<>();
+ this.createOrderColumn = new LinkedList<>();
+ this.columnDrift = new LinkedList<>();
}
/**
@@ -196,7 +145,7 @@ public class CarbonTable implements Serializable, Writable {
* DataTypes are not converted to the appropriate child classes.
* This method will cast the same to the appropriate classes
*/
- public static void updateTableInfo(TableInfo tableInfo) {
+ private static void updateTableInfo(TableInfo tableInfo) {
List<DataMapSchema> dataMapSchemas = new ArrayList<>();
for (DataMapSchema dataMapSchema : tableInfo.getDataMapSchemaList()) {
DataMapSchema newDataMapSchema = DataMapSchemaFactory.INSTANCE
@@ -267,7 +216,7 @@ public class CarbonTable implements Serializable, Writable {
}
/**
- * @param tableInfo
+ * Build {@link CarbonTable} from a {@link TableInfo} object.
*/
public static CarbonTable buildFromTableInfo(TableInfo tableInfo) {
CarbonTable table = new CarbonTable();
@@ -276,12 +225,7 @@ public class CarbonTable implements Serializable, Writable {
}
/**
- * build table unique name
- * all should call this method to build table unique name
- *
- * @param databaseName
- * @param tableName
- * @return
+ * Return table unique name
*/
public static String buildUniqueName(String databaseName, String tableName) {
return (databaseName + CarbonCommonConstants.UNDERSCORE + tableName).toLowerCase(
@@ -290,10 +234,6 @@ public class CarbonTable implements Serializable, Writable {
/**
* Get Dimension for columnName from list of dimensions
- *
- * @param columnName
- * @param dimensions
- * @return
*/
public static CarbonDimension getCarbonDimension(String columnName,
List<CarbonDimension> dimensions) {
@@ -328,27 +268,19 @@ public class CarbonTable implements Serializable, Writable {
}
/**
- * update the carbon table by using the passed tableInfo
- *
- * @param table
- * @param tableInfo
+ * Update the carbon table by using the passed tableInfo
*/
public static void updateTableByTableInfo(CarbonTable table, TableInfo tableInfo) {
updateTableInfo(tableInfo);
table.tableInfo = tableInfo;
- table.blockSize = tableInfo.getTableBlockSizeInMB();
- table.tableLastUpdatedTime = tableInfo.getLastUpdatedTime();
- table.tableUniqueName = tableInfo.getTableUniqueName();
table.setTransactionalTable(tableInfo.isTransactionalTable());
table.fillDimensionsAndMeasuresForTables(tableInfo.getFactTable());
- table.fillCreateOrderColumn(tableInfo.getFactTable().getTableName());
+ table.fillCreateOrderColumn();
if (tableInfo.getFactTable().getBucketingInfo() != null) {
- table.tableBucketMap.put(tableInfo.getFactTable().getTableName(),
- tableInfo.getFactTable().getBucketingInfo());
+ table.bucket = tableInfo.getFactTable().getBucketingInfo();
}
if (tableInfo.getFactTable().getPartitionInfo() != null) {
- table.tablePartitionMap.put(tableInfo.getFactTable().getTableName(),
- tableInfo.getFactTable().getPartitionInfo());
+ table.partition = tableInfo.getFactTable().getPartitionInfo();
}
table.hasDataMapSchema =
null != tableInfo.getDataMapSchemaList() && tableInfo.getDataMapSchemaList().size() > 0;
@@ -358,9 +290,6 @@ public class CarbonTable implements Serializable, Writable {
/**
* This method sets whether the local dictionary is enabled or not, and the local dictionary
* threshold, if not defined default value are considered.
- *
- * @param table
- * @param tableInfo
*/
private static void setLocalDictInfo(CarbonTable table, TableInfo tableInfo) {
Map<String, String> tableProperties = tableInfo.getFactTable().getTableProperties();
@@ -385,16 +314,12 @@ public class CarbonTable implements Serializable, Writable {
}
/**
- * fill columns as per user provided order
- *
- * @param tableName
+ * Fill columns as per user provided order
*/
- private void fillCreateOrderColumn(String tableName) {
+ private void fillCreateOrderColumn() {
List<CarbonColumn> columns = new ArrayList<CarbonColumn>();
- List<CarbonDimension> dimensions = this.tableDimensionsMap.get(tableName);
- List<CarbonMeasure> measures = this.tableMeasuresMap.get(tableName);
- columns.addAll(dimensions);
- columns.addAll(measures);
+ columns.addAll(visibleDimensions);
+ columns.addAll(visibleMeasures);
Collections.sort(columns, new Comparator<CarbonColumn>() {
@Override
@@ -403,21 +328,17 @@ public class CarbonTable implements Serializable, Writable {
}
});
- this.createOrderColumn.put(tableName, columns);
+ this.createOrderColumn = columns;
}
/**
* Fill allDimensions and allMeasures for carbon table
- *
- * @param tableSchema
*/
private void fillDimensionsAndMeasuresForTables(TableSchema tableSchema) {
- List<CarbonDimension> primitiveDimensions = new ArrayList<CarbonDimension>();
List<CarbonDimension> implicitDimensions = new ArrayList<CarbonDimension>();
allDimensions = new ArrayList<CarbonDimension>();
allMeasures = new ArrayList<CarbonMeasure>();
- this.tablePrimitiveDimensionsMap.put(this.tableUniqueName, primitiveDimensions);
- this.tableImplicitDimensionsMap.put(tableSchema.getTableName(), implicitDimensions);
+ this.implicitDimensions = implicitDimensions;
int dimensionOrdinal = 0;
int measureOrdinal = 0;
int keyOrdinal = 0;
@@ -434,7 +355,7 @@ public class CarbonTable implements Serializable, Writable {
allDimensions.add(complexDimension);
dimensionOrdinal =
readAllComplexTypeChildrens(dimensionOrdinal, columnSchema.getNumberOfChild(),
- listOfColumns, complexDimension, primitiveDimensions);
+ listOfColumns, complexDimension);
i = dimensionOrdinal - 1;
complexTypeOrdinal = assignComplexOrdinal(complexDimension, complexTypeOrdinal);
} else {
@@ -448,12 +369,10 @@ public class CarbonTable implements Serializable, Writable {
this.numberOfNoDictSortColumns++;
}
allDimensions.add(dimension);
- primitiveDimensions.add(dimension);
} else if (columnSchema.getEncodingList().contains(Encoding.DICTIONARY)) {
CarbonDimension dimension = new CarbonDimension(columnSchema, dimensionOrdinal++,
columnSchema.getSchemaOrdinal(), keyOrdinal++, -1);
allDimensions.add(dimension);
- primitiveDimensions.add(dimension);
}
}
} else {
@@ -461,8 +380,8 @@ public class CarbonTable implements Serializable, Writable {
new CarbonMeasure(columnSchema, measureOrdinal++, columnSchema.getSchemaOrdinal()));
}
}
- fillVisibleDimensions(tableSchema.getTableName());
- fillVisibleMeasures(tableSchema.getTableName());
+ fillVisibleDimensions();
+ fillVisibleMeasures();
addImplicitDimension(dimensionOrdinal, implicitDimensions);
CarbonUtil.setLocalDictColumnsToWrapperSchema(tableSchema.getListOfColumns(),
tableSchema.getTableProperties(),
@@ -472,9 +391,6 @@ public class CarbonTable implements Serializable, Writable {
/**
* This method will add implicit dimension into carbontable
- *
- * @param dimensionOrdinal
- * @param dimensions
*/
private void addImplicitDimension(int dimensionOrdinal, List<CarbonDimension> dimensions) {
dimensions.add(new CarbonImplicitDimension(dimensionOrdinal,
@@ -485,27 +401,17 @@ public class CarbonTable implements Serializable, Writable {
/**
* to get the all dimension of a table
- *
- * @param tableName
- * @return
*/
- public List<CarbonDimension> getImplicitDimensionByTableName(String tableName) {
- return tableImplicitDimensionsMap.get(tableName);
+ public List<CarbonDimension> getImplicitDimensions() {
+ return implicitDimensions;
}
/**
* Read all primitive/complex children and set it as list of child carbon dimension to parent
* dimension
- *
- * @param dimensionOrdinal
- * @param childCount
- * @param listOfColumns
- * @param parentDimension
- * @return
*/
private int readAllComplexTypeChildrens(int dimensionOrdinal, int childCount,
- List<ColumnSchema> listOfColumns, CarbonDimension parentDimension,
- List<CarbonDimension> primitiveDimensions) {
+ List<ColumnSchema> listOfColumns, CarbonDimension parentDimension) {
for (int i = 0; i < childCount; i++) {
ColumnSchema columnSchema = listOfColumns.get(dimensionOrdinal);
if (columnSchema.isDimensionColumn()) {
@@ -517,13 +423,12 @@ public class CarbonTable implements Serializable, Writable {
parentDimension.getListOfChildDimensions().add(complexDimension);
dimensionOrdinal =
readAllComplexTypeChildrens(dimensionOrdinal, columnSchema.getNumberOfChild(),
- listOfColumns, complexDimension, primitiveDimensions);
+ listOfColumns, complexDimension);
} else {
CarbonDimension carbonDimension =
new CarbonDimension(columnSchema, dimensionOrdinal++, columnSchema.getSchemaOrdinal(),
-1, -1);
parentDimension.getListOfChildDimensions().add(carbonDimension);
- primitiveDimensions.add(carbonDimension);
}
}
}
@@ -573,13 +478,11 @@ public class CarbonTable implements Serializable, Writable {
* @return the tableUniqueName
*/
public String getTableUniqueName() {
- return tableUniqueName;
+ return tableInfo.getTableUniqueName();
}
/**
- * is local dictionary enabled for the table
- *
- * @return
+ * Return true if local dictionary enabled for the table
*/
public boolean isLocalDictionaryEnabled() {
return isLocalDictionaryEnabled;
@@ -587,10 +490,8 @@ public class CarbonTable implements Serializable, Writable {
/**
* set whether local dictionary enabled or not
- *
- * @param localDictionaryEnabled
*/
- public void setLocalDictionaryEnabled(boolean localDictionaryEnabled) {
+ private void setLocalDictionaryEnabled(boolean localDictionaryEnabled) {
isLocalDictionaryEnabled = localDictionaryEnabled;
}
@@ -603,10 +504,8 @@ public class CarbonTable implements Serializable, Writable {
/**
* set the local dictionary generation threshold
- *
- * @param localDictionaryThreshold
*/
- public void setLocalDictionaryThreshold(int localDictionaryThreshold) {
+ private void setLocalDictionaryThreshold(int localDictionaryThreshold) {
this.localDictionaryThreshold = localDictionaryThreshold;
}
@@ -635,78 +534,36 @@ public class CarbonTable implements Serializable, Writable {
* @return the tableLastUpdatedTime
*/
public long getTableLastUpdatedTime() {
- return tableLastUpdatedTime;
- }
-
- /**
- * to get the number of dimension present in the table
- *
- * @param tableName
- * @return number of dimension present the table
- */
- public int getNumberOfDimensions(String tableName) {
- return tableDimensionsMap.get(tableName).size();
- }
-
- /**
- * to get the number of allMeasures present in the table
- *
- * @param tableName
- * @return number of allMeasures present the table
- */
- public int getNumberOfMeasures(String tableName) {
- return tableMeasuresMap.get(tableName).size();
- }
-
- /**
- * to get the all dimension of a table
- *
- * @param tableName
- * @return all dimension of a table
- */
- public List<CarbonDimension> getDimensionByTableName(String tableName) {
- return tableDimensionsMap.get(tableName);
- }
-
- /**
- * to get the all measure of a table
- *
- * @param tableName
- * @return all measure of a table
- */
- public List<CarbonMeasure> getMeasureByTableName(String tableName) {
- return tableMeasuresMap.get(tableName);
+ return tableInfo.getLastUpdatedTime();
}
/**
- * Return all dimensions of the table
+ * Return all visible dimensions of the table
*/
- public List<CarbonDimension> getDimensions() {
- return tableDimensionsMap.get(getTableName());
+ public List<CarbonDimension> getVisibleDimensions() {
+ return visibleDimensions;
}
/**
- * Return all measure of the table
+ * Return all visible measure of the table
*/
- public List<CarbonMeasure> getMeasures() {
- return tableMeasuresMap.get(getTableName());
+ public List<CarbonMeasure> getVisibleMeasures() {
+ return visibleMeasures;
}
/**
* This will give user created order column
- *
- * @return
*/
- public List<CarbonColumn> getCreateOrderColumn(String tableName) {
- return createOrderColumn.get(tableName);
+ public List<CarbonColumn> getCreateOrderColumn() {
+ return createOrderColumn;
}
/**
* This method will give storage order column list
*/
- public List<CarbonColumn> getStreamStorageOrderColumn(String tableName) {
- List<CarbonDimension> dimensions = tableDimensionsMap.get(tableName);
- List<CarbonMeasure> measures = tableMeasuresMap.get(tableName);
+ public List<CarbonColumn> getStreamStorageOrderColumn() {
+ List<CarbonDimension> dimensions = visibleDimensions;
+ List<CarbonMeasure> measures = visibleMeasures;
List<CarbonColumn> columnList = new ArrayList<>(dimensions.size() + measures.size());
List<CarbonColumn> complexDimensionList = new ArrayList<>(dimensions.size());
for (CarbonColumn column : dimensions) {
@@ -726,15 +583,10 @@ public class CarbonTable implements Serializable, Writable {
}
/**
- * to get particular measure from a table
- *
- * @param tableName
- * @param columnName
- * @return
+ * Get particular measure
*/
- public CarbonMeasure getMeasureByName(String tableName, String columnName) {
- List<CarbonMeasure> measureList = tableMeasuresMap.get(tableName);
- for (CarbonMeasure measure : measureList) {
+ public CarbonMeasure getMeasureByName(String columnName) {
+ for (CarbonMeasure measure : visibleMeasures) {
if (measure.getColName().equalsIgnoreCase(columnName)) {
return measure;
}
@@ -743,15 +595,11 @@ public class CarbonTable implements Serializable, Writable {
}
/**
- * to get particular dimension from a table
- *
- * @param tableName
- * @param columnName
- * @return
+ * Get particular dimension
*/
- public CarbonDimension getDimensionByName(String tableName, String columnName) {
+ public CarbonDimension getDimensionByName(String columnName) {
CarbonDimension carbonDimension = null;
- List<CarbonDimension> dimList = tableDimensionsMap.get(tableName);
+ List<CarbonDimension> dimList = visibleDimensions;
String[] colSplits = columnName.split("\\.");
StringBuffer tempColName = new StringBuffer(colSplits[0]);
for (String colSplit : colSplits) {
@@ -763,13 +611,13 @@ public class CarbonTable implements Serializable, Writable {
dimList = carbonDimension.getListOfChildDimensions();
}
}
- List<CarbonDimension> implicitDimList = tableImplicitDimensionsMap.get(tableName);
+ List<CarbonDimension> implicitDimList = implicitDimensions;
if (carbonDimension == null) {
carbonDimension = getCarbonDimension(columnName, implicitDimList);
}
if (colSplits.length > 1) {
- List<CarbonDimension> dimLists = tableDimensionsMap.get(tableName);
+ List<CarbonDimension> dimLists = visibleDimensions;
for (CarbonDimension dims : dimLists) {
if (dims.getColName().equalsIgnoreCase(colSplits[0])) {
// Set the parent Dimension
@@ -792,12 +640,10 @@ public class CarbonTable implements Serializable, Writable {
}
/**
- * @param tableName
- * @param columnName
- * @return
+ * @return column by column name
*/
- public CarbonColumn getColumnByName(String tableName, String columnName) {
- List<CarbonColumn> columns = createOrderColumn.get(tableName);
+ public CarbonColumn getColumnByName(String columnName) {
+ List<CarbonColumn> columns = createOrderColumn;
Iterator<CarbonColumn> colItr = columns.iterator();
while (colItr.hasNext()) {
CarbonColumn col = colItr.next();
@@ -809,29 +655,16 @@ public class CarbonTable implements Serializable, Writable {
}
/**
- * gets all children dimension for complex type
- *
- * @param dimName
- * @return list of child allDimensions
+ * Returns all children dimension for complex type
*/
public List<CarbonDimension> getChildren(String dimName) {
- for (List<CarbonDimension> list : tableDimensionsMap.values()) {
- List<CarbonDimension> childDims = getChildren(dimName, list);
- if (childDims != null) {
- return childDims;
- }
- }
- return null;
+ return getChildren(dimName, visibleDimensions);
}
/**
- * returns level 2 or more child allDimensions
- *
- * @param dimName
- * @param dimensions
- * @return list of child allDimensions
+ * Returns level 2 or more child allDimensions
*/
- public List<CarbonDimension> getChildren(String dimName, List<CarbonDimension> dimensions) {
+ private List<CarbonDimension> getChildren(String dimName, List<CarbonDimension> dimensions) {
for (CarbonDimension carbonDimension : dimensions) {
if (carbonDimension.getColName().equals(dimName)) {
return carbonDimension.getListOfChildDimensions();
@@ -847,28 +680,24 @@ public class CarbonTable implements Serializable, Writable {
return null;
}
- public BucketingInfo getBucketingInfo(String tableName) {
- return tableBucketMap.get(tableName);
+ public BucketingInfo getBucketingInfo() {
+ return bucket;
}
- public PartitionInfo getPartitionInfo(String tableName) {
- return tablePartitionMap.get(tableName);
+ public PartitionInfo getPartitionInfo() {
+ return partition;
}
public boolean isPartitionTable() {
- return null != tablePartitionMap.get(getTableName())
- && tablePartitionMap.get(getTableName()).getPartitionType() != PartitionType.NATIVE_HIVE;
+ return null != partition
+ && partition.getPartitionType() != PartitionType.NATIVE_HIVE;
}
public boolean isHivePartitionTable() {
- PartitionInfo partitionInfo = tablePartitionMap.get(getTableName());
+ PartitionInfo partitionInfo = partition;
return null != partitionInfo && partitionInfo.getPartitionType() == PartitionType.NATIVE_HIVE;
}
- public PartitionInfo getPartitionInfo() {
- return tablePartitionMap.get(getTableName());
- }
-
/**
* @return absolute table identifier
*/
@@ -887,16 +716,8 @@ public class CarbonTable implements Serializable, Writable {
return tableInfo.getOrCreateAbsoluteTableIdentifier().getCarbonTableIdentifier();
}
- /**
- * gets partition count for this table
- * TODO: to be implemented while supporting partitioning
- */
- public int getPartitionCount() {
- return 1;
- }
-
public int getBlockSizeInMB() {
- return blockSize;
+ return tableInfo.getTableBlockSizeInMB();
}
public int getBlockletSizeInMB() {
@@ -913,10 +734,9 @@ public class CarbonTable implements Serializable, Writable {
*
* @return primitive dimension of a table
*/
- public CarbonDimension getPrimitiveDimensionByName(String columnName) {
- List<CarbonDimension> dimList = tablePrimitiveDimensionsMap.get(tableUniqueName);
- for (CarbonDimension dim : dimList) {
- if (!dim.isInvisible() && dim.getColName().equalsIgnoreCase(columnName)) {
+ public CarbonColumn getPrimitiveDimensionByName(String columnName) {
+ for (CarbonDimension dim : visibleDimensions) {
+ if (dim.getNumberOfChild() == 0 && dim.getColName().equalsIgnoreCase(columnName)) {
return dim;
}
}
@@ -925,8 +745,6 @@ public class CarbonTable implements Serializable, Writable {
/**
* return all allDimensions in the table
- *
- * @return
*/
public List<CarbonDimension> getAllDimensions() {
return allDimensions;
@@ -934,10 +752,8 @@ public class CarbonTable implements Serializable, Writable {
/**
* This method will all the visible allDimensions
- *
- * @param tableName
*/
- private void fillVisibleDimensions(String tableName) {
+ private void fillVisibleDimensions() {
List<CarbonDimension> visibleDimensions = new ArrayList<CarbonDimension>(allDimensions.size());
for (CarbonDimension dimension : allDimensions) {
if (!dimension.isInvisible()) {
@@ -950,13 +766,11 @@ public class CarbonTable implements Serializable, Writable {
}
}
}
- tableDimensionsMap.put(tableName, visibleDimensions);
+ this.visibleDimensions = visibleDimensions;
}
/**
* return all allMeasures in the table
- *
- * @return
*/
public List<CarbonMeasure> getAllMeasures() {
return allMeasures;
@@ -972,38 +786,28 @@ public class CarbonTable implements Serializable, Writable {
/**
* This method will all the visible allMeasures
- *
- * @param tableName
*/
- private void fillVisibleMeasures(String tableName) {
+ private void fillVisibleMeasures() {
List<CarbonMeasure> visibleMeasures = new ArrayList<CarbonMeasure>(allMeasures.size());
for (CarbonMeasure measure : allMeasures) {
if (!measure.isInvisible()) {
visibleMeasures.add(measure);
}
}
- tableMeasuresMap.put(tableName, visibleMeasures);
+ this.visibleMeasures = visibleMeasures;
}
/**
- * Method to get the list of sort columns
- *
- * @param tableName
- * @return List of Sort column
+ * Get the list of sort columns
*/
- public List<String> getSortColumns(String tableName) {
- List<String> sort_columsList = new ArrayList<String>(allDimensions.size());
- List<CarbonDimension> carbonDimensions = tableDimensionsMap.get(tableName);
- for (CarbonDimension dim : carbonDimensions) {
+ public List<String> getSortColumns() {
+ List<String> sortColumnsList = new ArrayList<String>(allDimensions.size());
+ for (CarbonDimension dim : visibleDimensions) {
if (dim.isSortColumn()) {
- sort_columsList.add(dim.getColName());
+ sortColumnsList.add(dim.getColName());
}
}
- return sort_columsList;
- }
-
- public List<String> getSortColumns() {
- return getSortColumns(getTableName());
+ return sortColumnsList;
}
public int getNumberOfSortColumns() {
@@ -1020,7 +824,7 @@ public class CarbonTable implements Serializable, Writable {
if (rangeColumn == null) {
return null;
} else {
- return getColumnByName(getTableName(), rangeColumn);
+ return getColumnByName(rangeColumn);
}
}
@@ -1116,13 +920,21 @@ public class CarbonTable implements Serializable, Writable {
return dataSize + indexSize;
}
+ /**
+ * Return true if this is a transactional table.
+ * Transactional table means carbon will provide transactional support when user doing data
+ * management like data loading, whether it is success or failure, data will be in consistent
+ * state.
+ * The difference between Transactional and non Transactional table is
+ * non Transactional Table will not contain any Metadata folder and subsequently
+ * no TableStatus or Schema files.
+ */
public boolean isTransactionalTable() {
- return isTransactionalTable;
+ return tableInfo.isTransactionalTable();
}
public void setTransactionalTable(boolean transactionalTable) {
- isTransactionalTable = transactionalTable;
- getTableInfo().setTransactionalTable(transactionalTable);
+ tableInfo.setTransactionalTable(transactionalTable);
}
/**
@@ -1167,7 +979,7 @@ public class CarbonTable implements Serializable, Writable {
String[] columns = dataMapSchema.getIndexColumns();
List<CarbonColumn> indexColumn = new ArrayList<>(columns.length);
for (String column : columns) {
- CarbonColumn carbonColumn = getColumnByName(getTableName(), column.trim().toLowerCase());
+ CarbonColumn carbonColumn = getColumnByName(column.trim().toLowerCase());
if (carbonColumn == null) {
throw new MalformedDataMapCommandException(String
.format("column '%s' does not exist in table. Please check create DataMap statement.",
@@ -1208,19 +1020,16 @@ public class CarbonTable implements Serializable, Writable {
* Method to get the list of cached columns of the table.
* This method need to be used for Describe formatted like scenario where columns need to be
* displayed in the column create order
- *
- * @return
*/
public List<String> getMinMaxCachedColumnsInCreateOrder() {
List<String> cachedColsList = new ArrayList<>();
- String tableName = tableInfo.getFactTable().getTableName();
String cacheColumns =
tableInfo.getFactTable().getTableProperties().get(CarbonCommonConstants.COLUMN_META_CACHE);
if (null != cacheColumns) {
if (!cacheColumns.isEmpty()) {
String[] cachedCols = cacheColumns.split(",");
for (String column : cachedCols) {
- CarbonColumn carbonColumn = getColumnByName(tableName, column);
+ CarbonColumn carbonColumn = getColumnByName(column);
if (null != carbonColumn && !carbonColumn.isInvisible()) {
cachedColsList.add(carbonColumn.getColName());
}
@@ -1240,12 +1049,9 @@ public class CarbonTable implements Serializable, Writable {
* In alter add column scenarios it can happen that the newly added columns are being cached
* which do not exist in already loaded data. In those cases newly added columns should not be
* cached for the already loaded data
- *
- * @return
*/
public List<CarbonColumn> getMinMaxCacheColumns(SegmentProperties segmentProperties) {
List<CarbonColumn> minMaxCachedColsList = null;
- String tableName = tableInfo.getFactTable().getTableName();
String cacheColumns =
tableInfo.getFactTable().getTableProperties().get(CarbonCommonConstants.COLUMN_META_CACHE);
if (null != cacheColumns) {
@@ -1256,7 +1062,7 @@ public class CarbonTable implements Serializable, Writable {
// add the columns in storage order: first normal dimensions, then complex dimensions
// and then measures
for (String column : cachedCols) {
- CarbonDimension dimension = getDimensionByName(tableName, column);
+ CarbonDimension dimension = getDimensionByName(column);
// if found in dimension then add to dimension else add to measures
if (null != dimension) {
CarbonDimension dimensionFromCurrentBlock =
@@ -1277,7 +1083,7 @@ public class CarbonTable implements Serializable, Writable {
minMaxCachedColsList.addAll(complexDimensions);
// search for measures columns and fill measures
for (String measureColumn : measureColumns) {
- CarbonMeasure measure = getMeasureByName(tableName, measureColumn);
+ CarbonMeasure measure = getMeasureByName(measureColumn);
if (null != measure) {
CarbonMeasure measureFromCurrentBlock =
segmentProperties.getMeasureFromCurrentBlock(measure);
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
index 1a94608..d604e15 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModel.java
@@ -210,8 +210,8 @@ public class QueryModel {
// corresponding segment. So the filter column may not be present in it. so generate the
// dimension and measure from the carbontable
CarbonDimension dimension =
- table.getDimensionByName(table.getTableName(), col.getColumnName());
- CarbonMeasure measure = table.getMeasureByName(table.getTableName(), col.getColumnName());
+ table.getDimensionByName(col.getColumnName());
+ CarbonMeasure measure = table.getMeasureByName(col.getColumnName());
col.setDimension(dimension);
col.setMeasure(measure);
col.setCarbonColumn(dimension == null ? measure : dimension);
diff --git a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModelBuilder.java b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModelBuilder.java
index 61c9b5a..4a0af29 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModelBuilder.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/model/QueryModelBuilder.java
@@ -60,7 +60,7 @@ public class QueryModelBuilder {
int i = 0;
for (String projectionColumnName : projectionColumns) {
- CarbonDimension dimension = table.getDimensionByName(factTableName, projectionColumnName);
+ CarbonDimension dimension = table.getDimensionByName(projectionColumnName);
if (dimension != null) {
CarbonDimension complexParentDimension = dimension.getComplexParentDimension();
if (null != complexParentDimension && dimension.hasEncoding(Encoding.DICTIONARY)) {
@@ -73,7 +73,7 @@ public class QueryModelBuilder {
i++;
}
} else {
- CarbonMeasure measure = table.getMeasureByName(factTableName, projectionColumnName);
+ CarbonMeasure measure = table.getMeasureByName(projectionColumnName);
if (measure == null) {
throw new RuntimeException(
projectionColumnName + " column not found in the table " + factTableName);
@@ -191,7 +191,7 @@ public class QueryModelBuilder {
QueryProjection queryProjection = new QueryProjection();
int i = 0;
for (String projectionColumnName : projectionColumns) {
- CarbonDimension dimension = table.getDimensionByName(factTableName, projectionColumnName);
+ CarbonDimension dimension = table.getDimensionByName(projectionColumnName);
if (dimension != null) {
if (!mergedDimensions.contains(dimension)) {
if (!isAlreadyExists(dimension, queryProjection.getDimensions())) {
@@ -200,7 +200,7 @@ public class QueryModelBuilder {
}
}
} else {
- CarbonMeasure measure = table.getMeasureByName(factTableName, projectionColumnName);
+ CarbonMeasure measure = table.getMeasureByName(projectionColumnName);
if (measure == null) {
throw new RuntimeException(
projectionColumnName + " column not found in the table " + factTableName);
@@ -215,7 +215,7 @@ public class QueryModelBuilder {
private List<CarbonDimension> mergeChildColumns(List<Integer> childOrdinals) {
// Check If children if they are in the path of not.
List<CarbonDimension> mergedChild = new ArrayList<>();
- List<CarbonDimension> dimList = table.getDimensions();
+ List<CarbonDimension> dimList = table.getVisibleDimensions();
for (int i = 0; i < childOrdinals.size(); i++) {
for (int j = i; j < childOrdinals.size(); j++) {
CarbonDimension parentDimension = getDimensionBasedOnOrdinal(dimList, childOrdinals.get(i));
@@ -274,11 +274,11 @@ public class QueryModelBuilder {
public QueryModelBuilder projectAllColumns() {
QueryProjection projection = new QueryProjection();
- List<CarbonDimension> dimensions = table.getDimensions();
+ List<CarbonDimension> dimensions = table.getVisibleDimensions();
for (int i = 0; i < dimensions.size(); i++) {
projection.addDimension(dimensions.get(i), i);
}
- List<CarbonMeasure> measures = table.getMeasures();
+ List<CarbonMeasure> measures = table.getVisibleMeasures();
for (int i = 0; i < measures.size(); i++) {
projection.addMeasure(measures.get(i), i);
}
diff --git a/core/src/main/java/org/apache/carbondata/core/stream/StreamPruner.java b/core/src/main/java/org/apache/carbondata/core/stream/StreamPruner.java
index 178b9f1..3bb2d78 100644
--- a/core/src/main/java/org/apache/carbondata/core/stream/StreamPruner.java
+++ b/core/src/main/java/org/apache/carbondata/core/stream/StreamPruner.java
@@ -55,12 +55,12 @@ public class StreamPruner {
if (filterExp != null) {
// cache all columns
List<CarbonColumn> minMaxCacheColumns = new ArrayList<>();
- for (CarbonDimension dimension : carbonTable.getDimensions()) {
+ for (CarbonDimension dimension : carbonTable.getVisibleDimensions()) {
if (!dimension.isComplex()) {
minMaxCacheColumns.add(dimension);
}
}
- minMaxCacheColumns.addAll(carbonTable.getMeasures());
+ minMaxCacheColumns.addAll(carbonTable.getVisibleMeasures());
// prepare cardinality of all dimensions
List<ColumnSchema> listOfColumns =
carbonTable.getTableInfo().getFactTable().getListOfColumns();
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
index 34841e6..4d5de6b 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
@@ -3118,8 +3118,7 @@ public final class CarbonUtil {
public static Map<String, LocalDictionaryGenerator> getLocalDictionaryModel(
CarbonTable carbonTable) {
List<ColumnSchema> wrapperColumnSchema = CarbonUtil
- .getColumnSchemaList(carbonTable.getDimensionByTableName(carbonTable.getTableName()),
- carbonTable.getMeasureByTableName(carbonTable.getTableName()));
+ .getColumnSchemaList(carbonTable.getVisibleDimensions(), carbonTable.getVisibleMeasures());
boolean islocalDictEnabled = carbonTable.isLocalDictionaryEnabled();
// creates a map only if local dictionary is enabled, else map will be null
Map<String, LocalDictionaryGenerator> columnLocalDictGenMap = new HashMap<>();
diff --git a/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java b/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
index 76c0505..a33f2d4 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
@@ -37,6 +37,7 @@ import org.apache.carbondata.core.keygenerator.directdictionary.DirectDictionary
import org.apache.carbondata.core.metadata.datatype.DataType;
import org.apache.carbondata.core.metadata.datatype.DataTypes;
import org.apache.carbondata.core.metadata.encoder.Encoding;
+import org.apache.carbondata.core.metadata.schema.table.column.CarbonColumn;
import org.apache.carbondata.core.metadata.schema.table.column.CarbonDimension;
import org.apache.carbondata.core.metadata.schema.table.column.CarbonMeasure;
import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
@@ -767,7 +768,7 @@ public final class DataTypeUtil {
* data type
* @return
*/
- public static String normalizeColumnValueForItsDataType(String value, CarbonDimension dimension) {
+ public static String normalizeColumnValueForItsDataType(String value, CarbonColumn dimension) {
try {
Object parsedValue = null;
// validation will not be done for timestamp datatype as for timestamp direct dictionary
@@ -825,7 +826,7 @@ public final class DataTypeUtil {
return value;
}
- private static String parseStringToBigDecimal(String value, CarbonDimension dimension) {
+ private static String parseStringToBigDecimal(String value, CarbonColumn dimension) {
BigDecimal bigDecimal = new BigDecimal(value)
.setScale(dimension.getColumnSchema().getScale(), RoundingMode.HALF_UP);
BigDecimal normalizedValue =
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java
index 9bb5385..c219d2b 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/CarbonMetadataTest.java
@@ -79,13 +79,13 @@ public class CarbonMetadataTest {
@Test public void testGetCarbonTableReturingProperTableWithProperDimensionCount() {
int expectedResult = 1;
assertEquals(expectedResult,
- carbonMetadata.getCarbonTable(tableUniqueName).getNumberOfDimensions("carbontesttable"));
+ carbonMetadata.getCarbonTable(tableUniqueName).getVisibleDimensions().size());
}
@Test public void testGetCarbonTableReturingProperTableWithProperMeasureCount() {
int expectedResult = 1;
assertEquals(expectedResult,
- carbonMetadata.getCarbonTable(tableUniqueName).getNumberOfMeasures("carbontesttable"));
+ carbonMetadata.getCarbonTable(tableUniqueName).getVisibleMeasures().size());
}
@Test public void testGetCarbonTableReturingProperTableWithProperDatabaseName() {
@@ -175,7 +175,7 @@ public class CarbonMetadataTest {
return "carbonTestTable";
}
- @Mock public List<CarbonDimension> getDimensionByTableName(String tableName) {
+ @Mock public List<CarbonDimension> getVisibleDimensions() {
return carbonDimensions;
}
};
@@ -204,7 +204,7 @@ public class CarbonMetadataTest {
return "carbonTestTable";
}
- @Mock public List<CarbonDimension> getDimensionByTableName(String tableName) {
+ @Mock public List<CarbonDimension> getVisibleDimensions() {
return carbonDimensions;
}
};
@@ -246,7 +246,7 @@ public class CarbonMetadataTest {
return "carbonTestTable";
}
- @Mock public List<CarbonDimension> getDimensionByTableName(String tableName) {
+ @Mock public List<CarbonDimension> getVisibleDimensions() {
return carbonDimensions;
}
};
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
index d37aece..9c23012 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableTest.java
@@ -45,11 +45,11 @@ public class CarbonTableTest extends TestCase {
}
@Test public void testNumberOfDimensionReturnsProperCount() {
- assertEquals(1, carbonTable.getNumberOfDimensions("carbontesttable"));
+ assertEquals(1, carbonTable.getVisibleDimensions().size());
}
@Test public void testNumberOfMeasureReturnsProperCount() {
- assertEquals(1, carbonTable.getNumberOfMeasures("carbontesttable"));
+ assertEquals(1, carbonTable.getVisibleMeasures().size());
}
@Test public void testGetDatabaseNameResturnsDatabaseName() {
@@ -66,7 +66,7 @@ public class CarbonTableTest extends TestCase {
@Test public void testDimensionPresentInTableIsProper() {
CarbonDimension dimension = new CarbonDimension(getColumnarDimensionColumn(), 0, -1, -1);
- assertTrue(carbonTable.getDimensionByName("carbontesttable", "IMEI").equals(dimension));
+ assertTrue(carbonTable.getDimensionByName("IMEI").equals(dimension));
}
static ColumnSchema getColumnarDimensionColumn() {
diff --git a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
index a162622..93bea41 100644
--- a/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/metadata/schema/table/CarbonTableWithComplexTypesTest.java
@@ -44,11 +44,11 @@ public class CarbonTableWithComplexTypesTest extends TestCase {
}
@Test public void testNumberOfDimensionReturnsProperCount() {
- assertEquals(2, carbonTable.getNumberOfDimensions("carbontesttable"));
+ assertEquals(2, carbonTable.getVisibleDimensions().size());
}
@Test public void testNumberOfMeasureReturnsProperCount() {
- assertEquals(1, carbonTable.getNumberOfMeasures("carbontesttable"));
+ assertEquals(1, carbonTable.getVisibleMeasures().size());
}
@Test public void testGetDatabaseNameResturnsDatabaseName() {
diff --git a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVUtil.scala b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVUtil.scala
index 7b321b6..7e6ee02 100644
--- a/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVUtil.scala
+++ b/datamap/mv/core/src/main/scala/org/apache/carbondata/mv/datamap/MVUtil.scala
@@ -237,7 +237,7 @@ class MVUtil {
parentTableName: String,
parentDatabaseName: String,
carbonTable: CarbonTable): ColumnTableRelation = {
- val parentColumn = carbonTable.getColumnByName(parentTableName, parentColumnName)
+ val parentColumn = carbonTable.getColumnByName(parentColumnName)
var columnTableRelation: ColumnTableRelation = null
if (null != parentColumn) {
val parentColumnId = parentColumn.getColumnId
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
index fd5a7ed..2438890 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonFileInputFormat.java
@@ -131,7 +131,7 @@ public class CarbonFileInputFormat<T> extends CarbonInputFormat<T> implements Se
}
}
// this will be null in case of corrupt schema file.
- PartitionInfo partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName());
+ PartitionInfo partitionInfo = carbonTable.getPartitionInfo();
DataMapFilter filter = getFilterPredicates(job.getConfiguration());
// if external table Segments are found, add it to the List
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 74757ca..addce8d 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -176,7 +176,7 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
// process and resolve the expression
DataMapFilter dataMapFilter = getFilterPredicates(job.getConfiguration());
// this will be null in case of corrupt schema file.
- PartitionInfo partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName());
+ PartitionInfo partitionInfo = carbonTable.getPartitionInfo();
if (dataMapFilter != null) {
dataMapFilter.resolve(false);
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/StreamRecordReader.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/StreamRecordReader.java
index c272f42..020af65 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/StreamRecordReader.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/stream/StreamRecordReader.java
@@ -150,12 +150,12 @@ public class StreamRecordReader extends RecordReader<Void, Object> {
}
carbonTable = model.getTable();
List<CarbonDimension> dimensions =
- carbonTable.getDimensionByTableName(carbonTable.getTableName());
+ carbonTable.getVisibleDimensions();
dimensionCount = dimensions.size();
- List<CarbonMeasure> measures = carbonTable.getMeasureByTableName(carbonTable.getTableName());
+ List<CarbonMeasure> measures = carbonTable.getVisibleMeasures();
measureCount = measures.size();
List<CarbonColumn> carbonColumnList =
- carbonTable.getStreamStorageOrderColumn(carbonTable.getTableName());
+ carbonTable.getStreamStorageOrderColumn();
storageColumns = carbonColumnList.toArray(new CarbonColumn[carbonColumnList.size()]);
isNoDictColumn = CarbonDataProcessorUtil.getNoDictionaryMapping(storageColumns);
directDictionaryGenerators = new DirectDictionaryGenerator[storageColumns.length];
@@ -222,10 +222,8 @@ public class StreamRecordReader extends RecordReader<Void, Object> {
}
private void initializeFilter() {
-
List<ColumnSchema> wrapperColumnSchemaList = CarbonUtil
- .getColumnSchemaList(carbonTable.getDimensionByTableName(carbonTable.getTableName()),
- carbonTable.getMeasureByTableName(carbonTable.getTableName()));
+ .getColumnSchemaList(carbonTable.getVisibleDimensions(), carbonTable.getVisibleMeasures());
int[] dimLensWithComplex = new int[wrapperColumnSchemaList.size()];
for (int i = 0; i < dimLensWithComplex.length; i++) {
dimLensWithComplex[i] = Integer.MAX_VALUE;
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java
index b27e1df..1340be1 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java
@@ -383,7 +383,7 @@ public class StoreCreator {
private void writeDictionary(String factFilePath, CarbonTable table) throws Exception {
BufferedReader reader = new BufferedReader(new InputStreamReader(
new FileInputStream(factFilePath), "UTF-8"));
- List<CarbonDimension> dims = table.getDimensionByTableName(table.getTableName());
+ List<CarbonDimension> dims = table.getVisibleDimensions();
Set<String>[] set = new HashSet[dims.size()];
for (int i = 0; i < set.length; i++) {
set[i] = new HashSet<String>();
diff --git a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
index e6eba5b..5faab9c 100644
--- a/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
+++ b/integration/hive/src/main/java/org/apache/carbondata/hive/MapredCarbonInputFormat.java
@@ -176,7 +176,7 @@ public class MapredCarbonInputFormat extends CarbonTableInputFormat<ArrayWritabl
if (projection == null) {
projection = configuration.get("hive.io.file.readcolumn.names");
}
- List<CarbonColumn> carbonColumns = carbonTable.getCreateOrderColumn(tableName);
+ List<CarbonColumn> carbonColumns = carbonTable.getCreateOrderColumn();
List<String> carbonColumnNames = new ArrayList<>();
StringBuilder allColumns = new StringBuilder();
StringBuilder projectionColumns = new StringBuilder();
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
index 18c959e..165bd37 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
@@ -259,7 +259,7 @@ public class CarbonTableReader {
JobConf jobConf = new JobConf(config);
List<PartitionSpec> filteredPartitions = new ArrayList<>();
- PartitionInfo partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName());
+ PartitionInfo partitionInfo = carbonTable.getPartitionInfo();
LoadMetadataDetails[] loadMetadataDetails = null;
if (partitionInfo != null && partitionInfo.getPartitionType() == PartitionType.NATIVE_HIVE) {
try {
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
index 9172b31..639e178 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala
@@ -337,11 +337,9 @@ object CarbonDataStoreCreator {
new FileReader(factFilePath))
val header: String = reader.readLine()
val allCols: util.List[CarbonColumn] = new util.ArrayList[CarbonColumn]()
- val dimensions: util.List[CarbonDimension] =
- table.getDimensionByTableName(table.getTableName)
+ val dimensions: util.List[CarbonDimension] = table.getVisibleDimensions
allCols.addAll(dimensions)
- val msrs: List[CarbonMeasure] =
- table.getMeasureByTableName(table.getTableName)
+ val msrs: List[CarbonMeasure] = table.getVisibleMeasures
allCols.addAll(msrs)
val dimensionsIndex = dimensions.map(dim => dim.getColumnSchema.getSchemaOrdinal)
val dimensionSet: Array[util.List[String]] = Array.ofDim[util.List[String]](dimensions.size)
diff --git a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BucketingTestCase.scala b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BucketingTestCase.scala
index a6fcf62..32fcc70 100644
--- a/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BucketingTestCase.scala
+++ b/integration/spark-common-cluster-test/src/test/scala/org/apache/carbondata/cluster/sdv/generated/BucketingTestCase.scala
@@ -64,7 +64,7 @@ class BucketingTestCase extends QueryTest with BeforeAndAfterAll {
"('DICTIONARY_INCLUDE'='ID','BUCKETNUMBER'='4', 'BUCKETCOLUMNS'='ID')")
sql(s"LOAD DATA INPATH '$resourcesPath/source.csv' INTO TABLE bucket_table")
val table: CarbonTable = CarbonMetadata.getInstance().getCarbonTable("default_bucket_table")
- if (table != null && table.getBucketingInfo("bucket_table") != null) {
+ if (table != null && table.getBucketingInfo != null) {
assert(true)
} else {
assert(false, "Bucketing info does not exist")
@@ -78,7 +78,7 @@ class BucketingTestCase extends QueryTest with BeforeAndAfterAll {
"('BUCKETNUMBER'='4', 'BUCKETCOLUMNS'='name,phonetype')")
sql(s"LOAD DATA INPATH '$resourcesPath/source.csv' INTO TABLE bucket_table")
val table: CarbonTable = CarbonMetadata.getInstance().getCarbonTable("default_bucket_table")
- if (table != null && table.getBucketingInfo("bucket_table") != null) {
+ if (table != null && table.getBucketingInfo != null) {
assert(true)
} else {
assert(false, "Bucketing info does not exist")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
index eea450f..b11a57c 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
@@ -437,7 +437,7 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT, "false")
val table: CarbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "binaryTable")
- if (table != null && table.getBucketingInfo("binarytable") != null) {
+ if (table != null && table.getBucketingInfo() != null) {
assert(true)
} else {
assert(false, "Bucketing info does not exist")
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCreateTableWithDouble.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCreateTableWithDouble.scala
index 008ec6a..f08aa20 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCreateTableWithDouble.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/complexType/TestCreateTableWithDouble.scala
@@ -77,7 +77,7 @@ class TestCreateTableWithDouble extends QueryTest with BeforeAndAfterAll {
CarbonCommonConstants.DATABASE_DEFAULT_NAME, "doubleComplex2".toLowerCase(), "uniqueid")
val carbonTable =
CarbonMetadata.getInstance().getCarbonTable(tableIdentifier.getTableUniqueName)
- val dimExist = carbonTable.getDimensionByTableName("doubleComplex2".toLowerCase()).toArray.
+ val dimExist = carbonTable.getVisibleDimensions().toArray.
exists(_.asInstanceOf[CarbonDimension].getColName.equalsIgnoreCase("number"))
assertResult(dimExist)(true)
// assert that load and query is successful
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestNoInvertedIndexLoadAndQuery.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestNoInvertedIndexLoadAndQuery.scala
index f483827..0b0a321 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestNoInvertedIndexLoadAndQuery.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/integration/spark/testsuite/dataload/TestNoInvertedIndexLoadAndQuery.scala
@@ -312,11 +312,11 @@ class TestNoInvertedIndexLoadAndQuery extends QueryTest with BeforeAndAfterAll {
LOAD DATA LOCAL INPATH '$testData1' into table index1
""")
val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "index1")
- assert(carbonTable.getColumnByName("index1", "city").getColumnSchema.getEncodingList
+ assert(carbonTable.getColumnByName("city").getColumnSchema.getEncodingList
.contains(Encoding.INVERTED_INDEX))
- assert(carbonTable.getColumnByName("index1", "name").getColumnSchema.getEncodingList
+ assert(carbonTable.getColumnByName("name").getColumnSchema.getEncodingList
.contains(Encoding.INVERTED_INDEX))
- assert(!carbonTable.getColumnByName("index1", "id").getColumnSchema.getEncodingList
+ assert(!carbonTable.getColumnByName("id").getColumnSchema.getEncodingList
.contains(Encoding.INVERTED_INDEX))
checkAnswer(
sql(
@@ -336,11 +336,11 @@ class TestNoInvertedIndexLoadAndQuery extends QueryTest with BeforeAndAfterAll {
TBLPROPERTIES('INVERTED_INDEX'='city,name,id','SORT_COLUMNS'='city,name,id')
""")
val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "index1")
- assert(carbonTable.getColumnByName("index1", "city").getColumnSchema.getEncodingList
+ assert(carbonTable.getColumnByName("city").getColumnSchema.getEncodingList
.contains(Encoding.INVERTED_INDEX))
- assert(carbonTable.getColumnByName("index1", "name").getColumnSchema.getEncodingList
+ assert(carbonTable.getColumnByName("name").getColumnSchema.getEncodingList
.contains(Encoding.INVERTED_INDEX))
- assert(carbonTable.getColumnByName("index1", "id").getColumnSchema.getEncodingList
+ assert(carbonTable.getColumnByName("id").getColumnSchema.getEncodingList
.contains(Encoding.INVERTED_INDEX))
}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithColumnMetCacheAndCacheLevelProperty.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithColumnMetCacheAndCacheLevelProperty.scala
index 59a5a5e..5f003fe 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithColumnMetCacheAndCacheLevelProperty.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/allqueries/TestQueryWithColumnMetCacheAndCacheLevelProperty.scala
@@ -272,7 +272,7 @@ class TestQueryWithColumnMetCacheAndCacheLevelProperty extends QueryTest with Be
// form a filter expression and generate filter resolver tree
val columnExpression = new ColumnExpression("name", DataTypes.STRING)
columnExpression.setDimension(true)
- val dimension: CarbonDimension = carbonTable.getDimensionByName(carbonTable.getTableName, "name")
+ val dimension: CarbonDimension = carbonTable.getDimensionByName("name")
columnExpression.setDimension(dimension)
columnExpression.setCarbonColumn(dimension)
val literalValueExpression = new LiteralExpression("a", DataTypes.STRING)
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala
index 9719cfc..96f1095 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/longstring/VarcharDataTypesBasicTestCase.scala
@@ -326,9 +326,9 @@ class VarcharDataTypesBasicTestCase extends QueryTest with BeforeAndAfterEach wi
val dmTableName = longStringTable + "_" + datamapName
val dmTable = CarbonMetadata.getInstance().getCarbonTable("default", dmTableName)
assert(null != dmTable)
- assert(dmTable.getColumnByName(dmTableName.toLowerCase(), longStringTable + "_description").getDataType
+ assert(dmTable.getColumnByName(longStringTable + "_description").getDataType
== DataTypes.VARCHAR)
- assert(dmTable.getColumnByName(dmTableName.toLowerCase(), longStringTable + "_note").getDataType
+ assert(dmTable.getColumnByName(longStringTable + "_note").getDataType
== DataTypes.VARCHAR)
sql(s"DROP DATAMAP IF EXISTS $datamapName ON TABLE $longStringTable")
}
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
index 7322b95..b246839 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTable.scala
@@ -51,7 +51,7 @@ class TestDDLForPartitionTable extends QueryTest with BeforeAndAfterAll {
""".stripMargin)
val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "hashTable")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
assert(partitionInfo != null)
assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("empno"))
assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.INT)
@@ -74,7 +74,7 @@ class TestDDLForPartitionTable extends QueryTest with BeforeAndAfterAll {
""".stripMargin)
val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "rangeTable")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
assert(partitionInfo != null)
assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("doj"))
assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.TIMESTAMP)
@@ -100,7 +100,7 @@ class TestDDLForPartitionTable extends QueryTest with BeforeAndAfterAll {
| 'LIST_INFO'='0, 1, (2, 3)')
""".stripMargin)
val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "listTable")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
assert(partitionInfo != null)
assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("workgroupcategory"))
assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.STRING)
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
index ac2376a..0633d7f 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestDDLForPartitionTableWithDefaultProperties.scala
@@ -46,7 +46,7 @@ class TestDDLForPartitionTableWithDefaultProperties extends QueryTest with Befo
""".stripMargin)
val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "hashTable")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
assert(partitionInfo != null)
assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("empno"))
assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.INT)
@@ -69,7 +69,7 @@ class TestDDLForPartitionTableWithDefaultProperties extends QueryTest with Befo
""".stripMargin)
val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "rangeTable")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
assert(partitionInfo != null)
assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("doj"))
assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.TIMESTAMP)
@@ -96,7 +96,7 @@ class TestDDLForPartitionTableWithDefaultProperties extends QueryTest with Befo
| 'DICTIONARY_INCLUDE'='projectenddate')
""".stripMargin)
val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "listTable")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
assert(partitionInfo != null)
assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("projectenddate"))
assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.TIMESTAMP)
@@ -127,7 +127,7 @@ class TestDDLForPartitionTableWithDefaultProperties extends QueryTest with Befo
| 'LIST_INFO'='2017-06-11 , 2017-06-13')
""".stripMargin)
val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "listTableDate")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
assert(partitionInfo != null)
assert(partitionInfo.getColumnSchemaList.get(0).getColumnName.equalsIgnoreCase("projectenddate"))
assert(partitionInfo.getColumnSchemaList.get(0).getDataType == DataTypes.DATE)
diff --git a/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/Util.java b/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/Util.java
index 0f71f8c..2da5db3 100644
--- a/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/Util.java
+++ b/integration/spark-common/src/main/java/org/apache/carbondata/spark/util/Util.java
@@ -85,7 +85,7 @@ public class Util {
}
public static StructType convertToSparkSchema(CarbonTable table) {
- List<CarbonColumn> columns = table.getCreateOrderColumn(table.getTableName());
+ List<CarbonColumn> columns = table.getCreateOrderColumn();
ColumnSchema[] schema = new ColumnSchema[columns.size()];
int i = 0;
for (CarbonColumn column : columns) {
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala
index fd2cd19..33d5eaa 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/AlterTableLoadPartitionRDD.scala
@@ -44,9 +44,7 @@ class AlterTableLoadPartitionRDD[K, V](alterPartitionModel: AlterPartitionModel,
val segmentId = alterPartitionModel.segmentId
val oldPartitionIds = alterPartitionModel.oldPartitionIds
val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
- val databaseName = carbonTable.getDatabaseName
- val factTableName = carbonTable.getTableName
- val partitionInfo = carbonTable.getPartitionInfo(factTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
override protected def internalGetPartitions: Array[Partition] = {
val sc = alterPartitionModel.sqlContext.sparkContext
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
index 241720a..8941636 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/CarbonScanPartitionRDD.scala
@@ -80,8 +80,7 @@ class CarbonScanPartitionRDD(alterPartitionModel: AlterPartitionModel,
val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
val dimensions = carbonTable.getAllDimensions.asScala
val measures = carbonTable.getAllMeasures.asScala
- val partitionInfo = carbonTable
- .getPartitionInfo(absoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
val partitionColumn = partitionInfo.getColumnSchemaList().get(0)
val partitionDataType = partitionColumn.getDataType
val partitionColumnName = partitionColumn.getColumnName
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
index 4d42446..8861863 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/NewCarbonDataLoadRDD.scala
@@ -344,11 +344,11 @@ class NewRddIterator(rddIter: Iterator[Row],
carbonLoadModel.getSerializationNullFormat.split(CarbonCommonConstants.COMMA, 2)(1)
import scala.collection.JavaConverters._
private val isVarcharTypeMapping =
- carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getCreateOrderColumn(
- carbonLoadModel.getTableName).asScala.map(_.getDataType == DataTypes.VARCHAR)
+ carbonLoadModel.getCarbonDataLoadSchema
+ .getCarbonTable.getCreateOrderColumn.asScala.map(_.getDataType == DataTypes.VARCHAR)
private val isComplexTypeMapping =
- carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable.getCreateOrderColumn(
- carbonLoadModel.getTableName).asScala.map(_.isComplex())
+ carbonLoadModel.getCarbonDataLoadSchema
+ .getCarbonTable.getCreateOrderColumn.asScala.map(_.isComplex())
def hasNext: Boolean = rddIter.hasNext
def next: Array[AnyRef] = {
@@ -399,7 +399,7 @@ class LazyRddIterator(serializer: SerializerInstance,
import scala.collection.JavaConverters._
private val isVarcharTypeMapping = {
val col2VarcharType = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
- .getCreateOrderColumn(carbonLoadModel.getTableName).asScala
+ .getCreateOrderColumn().asScala
.map(c => c.getColName -> (c.getDataType == DataTypes.VARCHAR)).toMap
carbonLoadModel.getCsvHeaderColumns.map(c => {
val r = col2VarcharType.get(c.toLowerCase)
@@ -463,7 +463,7 @@ class PartitionTableDataLoaderRDD[K, V](
val executionErrors = new ExecutionErrors(FailureCauses.NONE, "")
val model: CarbonLoadModel = carbonLoadModel
val carbonTable = model.getCarbonDataLoadSchema.getCarbonTable
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
val uniqueLoadStatusId =
carbonLoadModel.getTableName + CarbonCommonConstants.UNDERSCORE + theSplit.index
try {
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
index e215ab0..bc90b89 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionDropper.scala
@@ -43,11 +43,11 @@ object PartitionDropper {
val dbName = carbonTable.getDatabaseName
val tableName = carbonTable.getTableName
val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
- val partitionInfo = carbonTable.getPartitionInfo(tableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
val partitioner = PartitionFactory.getPartitioner(partitionInfo)
var finalDropStatus = false
- val bucketInfo = carbonTable.getBucketingInfo(tableName)
+ val bucketInfo = carbonTable.getBucketingInfo()
val bucketNumber = bucketInfo match {
case null => 1
case _ => bucketInfo.getNumOfRanges
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
index f99b5e2..bd46c24 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/PartitionSplitter.scala
@@ -42,13 +42,13 @@ object PartitionSplitter {
val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
val tableName = carbonTable.getTableName
val databaseName = carbonTable.getDatabaseName
- val bucketInfo = carbonTable.getBucketingInfo(tableName)
+ val bucketInfo = carbonTable.getBucketingInfo()
var finalSplitStatus = false
val bucketNumber = bucketInfo match {
case null => 1
case _ => bucketInfo.getNumOfRanges
}
- val partitionInfo = carbonTable.getPartitionInfo(tableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
val partitioner = PartitionFactory.getPartitioner(partitionInfo)
for (i <- 0 until bucketNumber) {
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
index d754781..794e7d2 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/rdd/StreamHandoffRDD.scala
@@ -150,7 +150,7 @@ class StreamHandoffRDD[K, V](
CarbonInputFormat.setTableName(hadoopConf, carbonTable.getTableName)
CarbonInputFormat.setTablePath(hadoopConf, carbonTable.getTablePath)
val projection = new CarbonProjection
- val dataFields = carbonTable.getStreamStorageOrderColumn(carbonTable.getTableName)
+ val dataFields = carbonTable.getStreamStorageOrderColumn()
(0 until dataFields.size()).foreach { index =>
projection.addColumn(dataFields.get(index).getColName)
}
@@ -175,8 +175,7 @@ class StreamHandoffRDD[K, V](
carbonTable: CarbonTable
): CompactionResultSortProcessor = {
val wrapperColumnSchemaList = CarbonUtil.getColumnSchemaList(
- carbonTable.getDimensionByTableName(carbonTable.getTableName),
- carbonTable.getMeasureByTableName(carbonTable.getTableName))
+ carbonTable.getVisibleDimensions, carbonTable.getVisibleMeasures)
val dimLensWithComplex =
(0 until wrapperColumnSchemaList.size()).map(_ => Integer.MAX_VALUE).toArray
val dictionaryColumnCardinality =
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
index d94c5d7..5f9f9bb 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CarbonScalaUtil.scala
@@ -259,7 +259,7 @@ object CarbonScalaUtil {
} else {
pValue
}
- val carbonColumn = table.getColumnByName(table.getTableName, col.toLowerCase)
+ val carbonColumn = table.getColumnByName(col.toLowerCase)
val dataType =
CarbonSparkDataSourceUtil.convertCarbonToSparkDataType(carbonColumn.getDataType)
try {
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
index 8d6cdfb..f05689d 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala
@@ -963,7 +963,7 @@ object CommonUtil {
}
def validateSortColumns(carbonTable: CarbonTable, newProperties: Map[String, String]): Unit = {
- val fields = carbonTable.getCreateOrderColumn(carbonTable.getTableName).asScala
+ val fields = carbonTable.getCreateOrderColumn().asScala
val tableProperties = carbonTable.getTableInfo.getFactTable.getTableProperties
var sortKeyOption = newProperties.get(CarbonCommonConstants.SORT_COLUMNS)
val varcharColsString = tableProperties.get(CarbonCommonConstants.LONG_STRING_COLUMNS)
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
index c16d935..fb337cd 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/spark/util/GlobalDictionaryUtil.scala
@@ -628,8 +628,7 @@ object GlobalDictionaryUtil {
val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier
val dictfolderPath = CarbonTablePath.getMetadataPath(carbonLoadModel.getTablePath)
// columns which need to generate global dictionary file
- val dimensions = carbonTable.getDimensionByTableName(
- carbonTable.getTableName).asScala.toArray
+ val dimensions = carbonTable.getVisibleDimensions().asScala.toArray
// generate global dict from pre defined column dict file
carbonLoadModel.initPredefDictMap()
val allDictionaryPath = carbonLoadModel.getAllDictPath
diff --git a/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala b/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
index 2138580..2596e8a 100644
--- a/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
+++ b/integration/spark-common/src/main/scala/org/apache/carbondata/streaming/StreamSinkFactory.scala
@@ -250,7 +250,7 @@ object StreamSinkFactory {
val optionsFinal = LoadOption.fillOptionWithDefaultValue(parameters.asJava)
optionsFinal.put("sort_scope", "no_sort")
if (parameters.get("fileheader").isEmpty) {
- optionsFinal.put("fileheader", carbonTable.getCreateOrderColumn(carbonTable.getTableName)
+ optionsFinal.put("fileheader", carbonTable.getCreateOrderColumn()
.asScala.map(_.getColName).mkString(","))
}
optionsFinal
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
index 879addd..3db3ebe 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchemaCommon.scala
@@ -686,9 +686,8 @@ class TableNewProcessor(cm: TableModel) {
val field = cm.dimCols.find(keyDim equals _.column).get
val encoders = if (getEncoderFromParent(field)) {
cm.parentTable.get.getColumnByName(
- cm.parentTable.get.getTableName,
- cm.dataMapRelation.get.get(field).get.columnTableRelationList.
- get(0).parentColumnName).getEncoder
+ cm.dataMapRelation.get(field).columnTableRelationList.get.head.parentColumnName
+ ).getEncoder
} else {
val encoders = new java.util.ArrayList[Encoding]()
encoders.add(Encoding.DICTIONARY)
@@ -713,10 +712,8 @@ class TableNewProcessor(cm: TableModel) {
val sortField = cm.sortKeyDims.get.find(field.column equals _)
if (sortField.isEmpty) {
val encoders = if (getEncoderFromParent(field)) {
- cm.parentTable.get.getColumnByName(
- cm.parentTable.get.getTableName,
- cm.dataMapRelation.get.get(field).get.
- columnTableRelationList.get(0).parentColumnName).getEncoder
+ cm.parentTable.get.getColumnByName(cm.dataMapRelation.get.get(field).get.
+ columnTableRelationList.get(0).parentColumnName).getEncoder
} else {
val encoders = new java.util.ArrayList[Encoding]()
encoders.add(Encoding.DICTIONARY)
@@ -759,14 +756,13 @@ class TableNewProcessor(cm: TableModel) {
// same encoder can be applied on aggregate table
val encoders = if (getEncoderFromParent(field)) {
isAggFunPresent =
- cm.dataMapRelation.get.get(field).get.aggregateFunction.equalsIgnoreCase("sum") ||
- cm.dataMapRelation.get.get(field).get.aggregateFunction.equals("avg") ||
- cm.dataMapRelation.get.get(field).get.aggregateFunction.equals("count")
+ cm.dataMapRelation.get(field).aggregateFunction.equalsIgnoreCase("sum") ||
+ cm.dataMapRelation.get(field).aggregateFunction.equals("avg") ||
+ cm.dataMapRelation.get(field).aggregateFunction.equals("count")
if (!isAggFunPresent) {
cm.parentTable.get.getColumnByName(
- cm.parentTable.get.getTableName,
- cm.dataMapRelation.get.get(field).get.columnTableRelationList.get(0).parentColumnName)
- .getEncoder
+ cm.dataMapRelation.get(field).columnTableRelationList.get.head.parentColumnName
+ ).getEncoder
} else {
new java.util.ArrayList[Encoding]()
}
diff --git a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
index 4440e3a..ca187e8 100644
--- a/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
+++ b/integration/spark-common/src/main/scala/org/apache/spark/sql/execution/streaming/CarbonAppendableStreamSink.scala
@@ -113,7 +113,7 @@ class CarbonAppendableStreamSink(
carbonLoadModel
.getCarbonDataLoadSchema
.getCarbonTable
- .getMeasures
+ .getVisibleMeasures
.asScala
.map(_.getDataType)
.toArray
@@ -264,7 +264,7 @@ object CarbonAppendableStreamSink {
val rowSchema = queryExecution.analyzed.schema
val isVarcharTypeMapping = {
val col2VarcharType = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
- .getCreateOrderColumn(carbonLoadModel.getTableName).asScala
+ .getCreateOrderColumn().asScala
.map(c => c.getColName -> (c.getDataType == DataTypes.VARCHAR)).toMap
rowSchema.fieldNames.map(c => {
val r = col2VarcharType.get(c.toLowerCase)
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
index 383100f..5ee5982 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/datamap/IndexDataMapRebuildRDD.scala
@@ -184,7 +184,7 @@ class RawBytesReadSupport(segmentProperties: SegmentProperties, indexColumns: Ar
val columnPartitioner = new util.ArrayList[Integer](dictIndexColumns.length)
dictIndexColumns.foreach { col =>
- val dim = carbonTable.getDimensionByName(carbonTable.getTableName, col.getColName)
+ val dim = carbonTable.getDimensionByName(col.getColName)
val currentBlockDimension = segmentProperties.getDimensionFromCurrentBlock(dim)
if (null != currentBlockDimension) {
columnCardinality.add(segmentProperties.getDimColumnsCardinality.apply(
@@ -226,7 +226,7 @@ class RawBytesReadSupport(segmentProperties: SegmentProperties, indexColumns: Ar
// prepare index info to extract data from query result
indexColumns.foreach { col =>
if (col.isDimension) {
- val dim = carbonTable.getDimensionByName(carbonTable.getTableName, col.getColName)
+ val dim = carbonTable.getDimensionByName(col.getColName)
if (!dim.isGlobalDictionaryEncoding && !dim.isDirectDictionaryEncoding) {
indexCol2IdxInNoDictArray =
indexCol2IdxInNoDictArray + (col.getColName -> indexCol2IdxInNoDictArray.size)
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
index 6767a5c..fa7b9f5 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonDataRDDFactory.scala
@@ -368,7 +368,7 @@ object CarbonDataRDDFactory {
}
}
} else {
- status = if (carbonTable.getPartitionInfo(carbonTable.getTableName) != null) {
+ status = if (carbonTable.getPartitionInfo() != null) {
loadDataForPartitionTable(sqlContext, dataFrame, carbonLoadModel, hadoopConf)
} else if (dataFrame.isEmpty && isSortTable &&
carbonLoadModel.getRangePartitionColumn != null &&
@@ -994,7 +994,7 @@ object CarbonDataRDDFactory {
carbonLoadModel: CarbonLoadModel,
hadoopConf: Configuration): RDD[Row] = {
val carbonTable = carbonLoadModel.getCarbonDataLoadSchema.getCarbonTable
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
val partitionColumn = partitionInfo.getColumnSchemaList.get(0).getColumnName
val partitionColumnDataType = partitionInfo.getColumnSchemaList.get(0).getDataType
val columns = carbonLoadModel.getCsvHeaderColumns
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
index 0fed2ef..0eef9dc 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
@@ -402,7 +402,7 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
segments: Array[Segment]
): DataFrame = {
val columns = carbonTable
- .getCreateOrderColumn(carbonTable.getTableName)
+ .getCreateOrderColumn()
.asScala
.map(_.getColName)
.toArray
@@ -457,8 +457,7 @@ class CarbonTableCompactor(carbonLoadModel: CarbonLoadModel,
CarbonTableOutputFormat.setDatabaseName(conf, carbonTable.getDatabaseName)
CarbonTableOutputFormat.setTableName(conf, carbonTable.getTableName)
CarbonTableOutputFormat.setCarbonTable(conf, carbonTable)
- val fieldList = carbonTable
- .getCreateOrderColumn(carbonTable.getTableName)
+ val fieldList = carbonTable.getCreateOrderColumn
.asScala
.map { column =>
new StructField(column.getColName, column.getDataType)
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala b/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
index 8feb1b9..867566b 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
@@ -37,12 +37,10 @@ case class TransformHolder(rdd: Any, mataData: CarbonMetaData)
object CarbonSparkUtil {
def createSparkMeta(carbonTable: CarbonTable): CarbonMetaData = {
- val dimensionsAttr = carbonTable.getDimensionByTableName(carbonTable.getTableName)
- .asScala.map(x => x.getColName) // wf : may be problem
- val measureAttr = carbonTable.getMeasureByTableName(carbonTable.getTableName)
- .asScala.map(x => x.getColName)
+ val dimensionsAttr = carbonTable.getVisibleDimensions.asScala.map(x => x.getColName)
+ val measureAttr = carbonTable.getVisibleMeasures.asScala.map(x => x.getColName)
val dictionary =
- carbonTable.getDimensionByTableName(carbonTable.getTableName).asScala.map { f =>
+ carbonTable.getVisibleDimensions.asScala.map { f =>
(f.getColName.toLowerCase,
f.hasEncoding(Encoding.DICTIONARY) && !f.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
!f.getDataType.isComplexType)
@@ -98,13 +96,13 @@ object CarbonSparkUtil {
sortWith(_.getSchemaOrdinal < _.getSchemaOrdinal)
val columnList = columnSchemas.toList.asJava
carbonRelation.dimensionsAttr.foreach(attr => {
- val carbonColumn = carbonTable.getColumnByName(carbonRelation.tableName, attr.name)
+ val carbonColumn = carbonTable.getColumnByName(attr.name)
val columnComment = getColumnComment(carbonColumn)
fields(columnList.indexOf(carbonColumn.getColumnSchema)) =
'`' + attr.name + '`' + ' ' + attr.dataType.catalogString + columnComment
})
carbonRelation.measureAttr.foreach(msrAtrr => {
- val carbonColumn = carbonTable.getColumnByName(carbonRelation.tableName, msrAtrr.name)
+ val carbonColumn = carbonTable.getColumnByName(msrAtrr.name)
val columnComment = getColumnComment(carbonColumn)
fields(columnList.indexOf(carbonColumn.getColumnSchema)) =
'`' + msrAtrr.name + '`' + ' ' + msrAtrr.dataType.catalogString + columnComment
diff --git a/integration/spark2/src/main/scala/org/apache/carbondata/stream/StreamJobManager.scala b/integration/spark2/src/main/scala/org/apache/carbondata/stream/StreamJobManager.scala
index ece688e..a36f561 100644
--- a/integration/spark2/src/main/scala/org/apache/carbondata/stream/StreamJobManager.scala
+++ b/integration/spark2/src/main/scala/org/apache/carbondata/stream/StreamJobManager.scala
@@ -60,7 +60,7 @@ object StreamJobManager {
"('streaming' tblproperty is not 'sink' or 'true')")
}
if (validateQuerySchema) {
- val fields = sink.getCreateOrderColumn(sink.getTableName).asScala.map { column =>
+ val fields = sink.getCreateOrderColumn().asScala.map { column =>
StructField(
column.getColName,
CarbonSparkDataSourceUtil.convertCarbonToSparkDataType(column.getDataType))
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
index 969544e..6d833b9 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/CarbonDictionaryDecoder.scala
@@ -410,8 +410,7 @@ object CarbonDictionaryDecoder {
val relation = relations.find(p => p.contains(attr))
if (relation.isDefined && canBeDecoded(attr, profile)) {
val carbonTable = relation.get.carbonRelation.carbonRelation.metaData.carbonTable
- val carbonDimension = carbonTable
- .getDimensionByName(carbonTable.getTableName, attr.name)
+ val carbonDimension = carbonTable.getDimensionByName(attr.name)
if (carbonDimension != null &&
carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
!carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
@@ -442,8 +441,7 @@ object CarbonDictionaryDecoder {
val relation = relations.find(p => p.contains(attr))
if (relation.isDefined) {
val carbonTable = relation.get.carbonRelation.carbonRelation.metaData.carbonTable
- val carbonDimension = carbonTable
- .getDimensionByName(carbonTable.getTableName, attr.name)
+ val carbonDimension = carbonTable.getDimensionByName(attr.name)
if (carbonDimension != null &&
carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
!carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
@@ -487,7 +485,7 @@ object CarbonDictionaryDecoder {
if (relation.isDefined && CarbonDictionaryDecoder.canBeDecoded(attr, profile)) {
val carbonTable = relation.get.carbonRelation.carbonRelation.metaData.carbonTable
val carbonDimension =
- carbonTable.getDimensionByName(carbonTable.getTableName, attr.name)
+ carbonTable.getDimensionByName(attr.name)
if (carbonDimension != null &&
carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
!carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
@@ -542,7 +540,7 @@ class CarbonDecoderRDD(
if (relation.isDefined && canBeDecoded(attr)) {
val carbonTable = relation.get.carbonRelation.carbonRelation.metaData.carbonTable
val carbonDimension =
- carbonTable.getDimensionByName(carbonTable.getTableName, attr.name)
+ carbonTable.getDimensionByName(attr.name)
if (carbonDimension != null &&
carbonDimension.hasEncoding(Encoding.DICTIONARY) &&
!carbonDimension.hasEncoding(Encoding.DIRECT_DICTIONARY) &&
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
index 280eb12..72340f4 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonLoadDataCommand.scala
@@ -462,8 +462,7 @@ case class CarbonLoadDataCommand(
val carbonTableIdentifier = carbonTable.getAbsoluteTableIdentifier
.getCarbonTableIdentifier
val dictFolderPath = CarbonTablePath.getMetadataPath(carbonLoadModel.getTablePath)
- val dimensions = carbonTable.getDimensionByTableName(
- carbonTable.getTableName).asScala.toArray
+ val dimensions = carbonTable.getVisibleDimensions().asScala.toArray
val colDictFilePath = carbonLoadModel.getColDictFilePath
if (!StringUtils.isEmpty(colDictFilePath)) {
carbonLoadModel.initPredefDictMap()
@@ -673,8 +672,8 @@ case class CarbonLoadDataCommand(
// input data from csv files. Convert to logical plan
val allCols = new ArrayBuffer[String]()
// get only the visible dimensions from table
- allCols ++= table.getDimensionByTableName(table.getTableName).asScala.map(_.getColName)
- allCols ++= table.getMeasureByTableName(table.getTableName).asScala.map(_.getColName)
+ allCols ++= table.getVisibleDimensions().asScala.map(_.getColName)
+ allCols ++= table.getVisibleMeasures.asScala.map(_.getColName)
var attributes =
StructType(
allCols.filterNot(_.equals(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)).map(
@@ -934,7 +933,7 @@ case class CarbonLoadDataCommand(
attributes = attributes.map { attr =>
// Update attribute datatypes in case of dictionary columns, in case of dictionary columns
// datatype is always int
- val column = table.getColumnByName(table.getTableName, attr.name)
+ val column = table.getColumnByName(attr.name)
if (column.hasEncoding(Encoding.DICTIONARY)) {
CarbonToSparkAdapter.createAttributeReference(attr.name,
IntegerType,
@@ -983,7 +982,7 @@ case class CarbonLoadDataCommand(
CarbonProperties.getInstance().getGlobalSortRddStorageLevel))
}
val child = Project(output, LogicalRDD(attributes, updatedRdd)(sparkSession))
- val sortColumns = table.getSortColumns(table.getTableName)
+ val sortColumns = table.getSortColumns()
val sortPlan =
Sort(
output.filter(f => sortColumns.contains(f.name)).map(SortOrder(_, Ascending)),
@@ -1094,7 +1093,7 @@ case class CarbonLoadDataCommand(
operationContext: OperationContext): LogicalRelation = {
val table = loadModel.getCarbonDataLoadSchema.getCarbonTable
val metastoreSchema = StructType(catalogTable.schema.fields.map{f =>
- val column = table.getColumnByName(table.getTableName, f.name)
+ val column = table.getColumnByName(f.name)
if (column.hasEncoding(Encoding.DICTIONARY)) {
f.copy(dataType = IntegerType)
} else if (f.dataType == TimestampType || f.dataType == DateType) {
@@ -1112,7 +1111,7 @@ case class CarbonLoadDataCommand(
catalog.filterPartitions(Nil) // materialize all the partitions in memory
}
var partitionSchema =
- StructType(table.getPartitionInfo(table.getTableName).getColumnSchemaList.asScala.map(field =>
+ StructType(table.getPartitionInfo().getColumnSchemaList.asScala.map(field =>
metastoreSchema.fields.find(_.name.equalsIgnoreCase(field.getColumnName))).map(_.get))
val dataSchema =
StructType(metastoreSchema
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
index 750b09b..6792887 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonProjectForUpdateCommand.scala
@@ -81,7 +81,7 @@ private[sql] case class CarbonProjectForUpdateCommand(
setAuditTable(carbonTable)
setAuditInfo(Map("plan" -> plan.simpleString))
columns.foreach { col =>
- val dataType = carbonTable.getColumnByName(tableName, col).getColumnSchema.getDataType
+ val dataType = carbonTable.getColumnByName(col).getColumnSchema.getDataType
if (dataType.isComplexType) {
throw new UnsupportedOperationException("Unsupported operation on Complex data type")
}
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
index 507fe02..2a0d113 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableDropPartitionCommand.scala
@@ -62,7 +62,7 @@ case class CarbonAlterTableDropPartitionCommand(
val carbonMetaStore = CarbonEnv.getInstance(sparkSession).carbonMetaStore
val carbonTable = CarbonEnv.getCarbonTable(Option(dbName), tableName)(sparkSession)
val tablePath = carbonTable.getTablePath
- val partitionInfo = carbonTable.getPartitionInfo(tableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
if (partitionInfo == null) {
throwMetadataException(dbName, tableName, "table is not a partition table")
}
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
index 36ddce4..d880ddf 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonAlterTableSplitPartitionCommand.scala
@@ -74,7 +74,7 @@ case class CarbonAlterTableSplitPartitionCommand(
throwMetadataException(dbName, tableName, "table not found")
}
val carbonTable = relation.carbonTable
- val partitionInfo = carbonTable.getPartitionInfo(tableName)
+ val partitionInfo = carbonTable.getPartitionInfo
val partitionIds = partitionInfo.getPartitionIds.asScala.map(_.asInstanceOf[Int]).toList
// keep a copy of partitionIdList before update partitionInfo.
// will be used in partition data scan
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonShowCarbonPartitionsCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonShowCarbonPartitionsCommand.scala
index 0e57513..4bcdec4b 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonShowCarbonPartitionsCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/partition/CarbonShowCarbonPartitionsCommand.scala
@@ -40,8 +40,7 @@ private[sql] case class CarbonShowCarbonPartitionsCommand(
.lookupRelation(tableIdentifier)(sparkSession).asInstanceOf[CarbonRelation]
val carbonTable = relation.carbonTable
setAuditTable(carbonTable)
- val partitionInfo = carbonTable.getPartitionInfo(
- carbonTable.getAbsoluteTableIdentifier.getCarbonTableIdentifier.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo
if (partitionInfo == null) {
throwMetadataException(carbonTable.getDatabaseName, carbonTable.getTableName,
"SHOW PARTITIONS is not allowed on a table that is not partitioned")
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
index 480d1f7..1d4abcb 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/preaaggregate/PreAggregateUtil.scala
@@ -195,7 +195,7 @@ object PreAggregateUtil {
parentTableName: String,
parentDatabaseName: String,
carbonTable: CarbonTable) : ColumnTableRelation = {
- val parentColumnId = carbonTable.getColumnByName(parentTableName, parentColumnName).getColumnId
+ val parentColumnId = carbonTable.getColumnByName(parentColumnName).getColumnId
val columnTableRelation = ColumnTableRelation(parentColumnName = parentColumnName.toLowerCase(),
parentColumnId = parentColumnId,
parentTableName = parentTableName,
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
index 6b0d709..dc52e5d 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala
@@ -94,7 +94,7 @@ private[sql] case class CarbonAlterTableAddColumnCommand(
val thriftTable = schemaConverter
.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
// carbon columns based on schema order
- val carbonColumns = carbonTable.getCreateOrderColumn(carbonTable.getTableName).asScala
+ val carbonColumns = carbonTable.getCreateOrderColumn().asScala
.collect { case carbonColumn if !carbonColumn.isInvisible => carbonColumn.getColumnSchema }
// sort the new columns based on schema order
val sortedColsBasedActualSchemaOrder = newCols.sortBy(a => a.getSchemaOrdinal)
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableColRenameDataTypeChangeCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableColRenameDataTypeChangeCommand.scala
index d3d63eb..7e66d34 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableColRenameDataTypeChangeCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableColRenameDataTypeChangeCommand.scala
@@ -123,7 +123,7 @@ private[sql] case class CarbonAlterTableColRenameDataTypeChangeCommand(
.fireEvent(alterTableColRenameAndDataTypeChangePreEvent, operationContext)
val newColumnName = alterTableColRenameAndDataTypeChangeModel.newColumnName.toLowerCase
val oldColumnName = alterTableColRenameAndDataTypeChangeModel.columnName.toLowerCase
- val carbonColumns = carbonTable.getCreateOrderColumn(tableName).asScala.filter(!_.isInvisible)
+ val carbonColumns = carbonTable.getCreateOrderColumn().asScala.filter(!_.isInvisible)
if (!carbonColumns.exists(_.getColName.equalsIgnoreCase(oldColumnName))) {
throwMetadataException(dbName, tableName, s"Column does not exist: $oldColumnName")
}
@@ -278,7 +278,7 @@ private[sql] case class CarbonAlterTableColRenameDataTypeChangeCommand(
oldCarbonColumn: CarbonColumn): Unit = {
val schemaConverter = new ThriftWrapperSchemaConverterImpl
// get the carbon column in schema order
- val carbonColumns = carbonTable.getCreateOrderColumn(carbonTable.getTableName).asScala
+ val carbonColumns = carbonTable.getCreateOrderColumn().asScala
.collect { case carbonColumn if !carbonColumn.isInvisible => carbonColumn.getColumnSchema }
// get the schema ordinal of the column for which the dataType changed or column is renamed
val schemaOrdinal = carbonColumns.indexOf(carbonColumns
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
index 8a2e837..3a6f1f5 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableDropColumnCommand.scala
@@ -62,8 +62,8 @@ private[sql] case class CarbonAlterTableDropColumnCommand(
throw new MalformedCarbonCommandException(
"alter table drop column is not supported for index datamap")
}
- val partitionInfo = carbonTable.getPartitionInfo(tableName)
- val tableColumns = carbonTable.getCreateOrderColumn(tableName).asScala
+ val partitionInfo = carbonTable.getPartitionInfo()
+ val tableColumns = carbonTable.getCreateOrderColumn().asScala
if (partitionInfo != null) {
val partitionColumnSchemaList = partitionInfo.getColumnSchemaList.asScala
.map(_.getColumnName)
@@ -149,7 +149,7 @@ private[sql] case class CarbonAlterTableDropColumnCommand(
schemaEvolutionEntry,
tableInfo)(sparkSession)
// get the columns in schema order and filter the dropped column in the column set
- val cols = carbonTable.getCreateOrderColumn(carbonTable.getTableName).asScala
+ val cols = carbonTable.getCreateOrderColumn().asScala
.collect { case carbonColumn if !carbonColumn.isInvisible => carbonColumn.getColumnSchema }
.filterNot(column => delCols.contains(column))
// In case of spark2.2 and above and , when we call
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
index 7ea62da..9ebbcde 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/timeseries/TimeSeriesUtil.scala
@@ -49,7 +49,7 @@ object TimeSeriesUtil {
if (!eventTime.isDefined) {
throw new MalformedCarbonCommandException("event_time not defined in time series")
} else {
- val carbonColumn = parentTable.getColumnByName(parentTable.getTableName, eventTime.get.trim)
+ val carbonColumn = parentTable.getColumnByName(eventTime.get.trim)
if (carbonColumn.getDataType != DataTypes.TIMESTAMP) {
throw new MalformedCarbonCommandException(
"Timeseries event time is only supported on Timestamp " +
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
index c0d1605..b667a57 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonLateDecodeStrategy.scala
@@ -607,11 +607,11 @@ private[sql] class CarbonLateDecodeStrategy extends SparkStrategy {
private def getPartitioning(carbonTable: CarbonTable,
output: Seq[Attribute]): Partitioning = {
- val info: BucketingInfo = carbonTable.getBucketingInfo(carbonTable.getTableName)
+ val info: BucketingInfo = carbonTable.getBucketingInfo()
if (info != null) {
val cols = info.getListOfColumns.asScala
val sortColumn = carbonTable.
- getDimensionByTableName(carbonTable.getTableName).get(0).getColName
+ getVisibleDimensions().get(0).getColName
val numBuckets = info.getNumOfRanges
val bucketColumns = cols.flatMap { n =>
val attrRef = output.find(_.name.equalsIgnoreCase(n.getColumnName))
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
index d316309..b616c20 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonPreAggregateRules.scala
@@ -1803,7 +1803,7 @@ case class CarbonPreAggregateQueryRules(sparkSession: SparkSession) extends Rule
carbonTable: CarbonTable,
isFilterColumn: Boolean = false,
timeseriesFunction: String = ""): QueryColumn = {
- val columnSchema = carbonTable.getColumnByName(carbonTable.getTableName, columnName.toLowerCase)
+ val columnSchema = carbonTable.getColumnByName(columnName.toLowerCase)
if(null == columnSchema) {
null
} else {
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
index 16dc43a..7adc025 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/hive/CarbonRelation.scala
@@ -50,12 +50,9 @@ case class CarbonRelation(
}
val dimensionsAttr: Seq[AttributeReference] = {
- val sett = new LinkedHashSet(
- carbonTable.getDimensionByTableName(carbonTable.getTableName)
- .asScala.asJava)
+ val sett = new LinkedHashSet(carbonTable.getVisibleDimensions.asScala.asJava)
sett.asScala.toSeq.map(dim => {
- val dimval = metaData.carbonTable
- .getDimensionByName(metaData.carbonTable.getTableName, dim.getColName)
+ val dimval = metaData.carbonTable.getDimensionByName(dim.getColName)
val output: DataType = dimval.getDataType.getName.toLowerCase match {
case "array" =>
CarbonMetastoreTypes.toDataType(
@@ -81,9 +78,9 @@ case class CarbonRelation(
val measureAttr = {
val factTable = carbonTable.getTableName
new LinkedHashSet(
- carbonTable.getMeasureByTableName(carbonTable.getTableName).asScala.asJava).asScala.toSeq
+ carbonTable.getVisibleMeasures.asScala.asJava).asScala.toSeq
.map { x =>
- val metastoreType = metaData.carbonTable.getMeasureByName(factTable, x.getColName)
+ val metastoreType = metaData.carbonTable.getMeasureByName(x.getColName)
.getDataType.getName.toLowerCase match {
case "decimal" => "decimal(" + x.getPrecision + "," + x.getScale + ")"
case others => others
@@ -96,7 +93,7 @@ case class CarbonRelation(
}
override val output = {
- val columns = carbonTable.getCreateOrderColumn(carbonTable.getTableName)
+ val columns = carbonTable.getCreateOrderColumn()
.asScala
val partitionColumnSchemas = if (carbonTable.getPartitionInfo() != null) {
carbonTable.getPartitionInfo.getColumnSchemaList.asScala
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
index 5e7039f..6add24a 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/AlterTableUtil.scala
@@ -576,8 +576,7 @@ object AlterTableUtil {
CarbonCommonConstants.COLUMN_META_CACHE} is not allowed for child datamaps")
}
val schemaList: util.List[ColumnSchema] = CarbonUtil
- .getColumnSchemaList(carbonTable.getDimensionByTableName(carbonTable.getTableName),
- carbonTable.getMeasureByTableName(carbonTable.getTableName))
+ .getColumnSchemaList(carbonTable.getVisibleDimensions, carbonTable.getVisibleMeasures)
val tableColumns: Seq[String] = schemaList.asScala
.map(columnSchema => columnSchema.getColumnName)
CommonUtil
@@ -610,7 +609,7 @@ object AlterTableUtil {
val errorMsg = "range_column not support multiple columns"
throw new MalformedCarbonCommandException(errorMsg)
}
- val rangeColumn = carbonTable.getColumnByName(carbonTable.getTableName, rangeColumnProp)
+ val rangeColumn = carbonTable.getColumnByName(rangeColumnProp)
if (rangeColumn == null) {
throw new MalformedCarbonCommandException(
s"Table property ${ CarbonCommonConstants.RANGE_COLUMN }: ${ rangeColumnProp }" +
@@ -664,7 +663,7 @@ object AlterTableUtil {
cachedColumns: String): Unit = {
if (cachedColumns.nonEmpty) {
cachedColumns.split(",").foreach { column =>
- val dimension = carbonTable.getDimensionByName(carbonTable.getTableName, column)
+ val dimension = carbonTable.getDimensionByName(column)
if (null != dimension && dimension.isComplex) {
val errorMessage =
s"$column is a complex type column and complex type is not allowed for " +
@@ -799,8 +798,7 @@ object AlterTableUtil {
// By default all the columns in the table will be cached. This case is to compare all the
// table columns already cached to the newly specified cached columns
val schemaList: util.List[ColumnSchema] = CarbonUtil
- .getColumnSchemaList(carbonTable.getDimensionByTableName(carbonTable.getTableName),
- carbonTable.getMeasureByTableName(carbonTable.getTableName))
+ .getColumnSchemaList(carbonTable.getVisibleDimensions, carbonTable.getVisibleMeasures)
val tableColumns: Array[String] = schemaList.asScala
.map(columnSchema => columnSchema.getColumnName).toArray
compareColumns(tableColumns, newColumns)
diff --git a/integration/spark2/src/main/scala/org/apache/spark/util/DataMapUtil.scala b/integration/spark2/src/main/scala/org/apache/spark/util/DataMapUtil.scala
index 311a88c..ffbe765 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/util/DataMapUtil.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/util/DataMapUtil.scala
@@ -41,7 +41,7 @@ object DataMapUtil {
fieldRelationMap: scala.collection.mutable.LinkedHashMap[Field, DataMapField],
tableProperties: mutable.Map[String, String]): Unit = {
var neworder = Seq[String]()
- val parentOrder = parentTable.getSortColumns(parentTable.getTableName).asScala
+ val parentOrder = parentTable.getSortColumns().asScala
parentOrder.foreach(parentcol =>
fields.filter(col => fieldRelationMap(col).aggregateFunction.isEmpty &&
fieldRelationMap(col).columnTableRelationList.size == 1 &&
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
index 5aa0b2a..5bdf3b4c 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/testsuite/partition/TestAlterPartitionTable.scala
@@ -252,7 +252,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
sql("""ALTER TABLE list_table_area ADD PARTITION ('OutSpace', 'Hi')""".stripMargin)
val carbonTable = CarbonEnv
.getCarbonTable(Option("default"), "list_table_area")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
val partitionIds = partitionInfo.getPartitionIds
val list_info = partitionInfo.getListInfo
assert(partitionIds == List(0, 1, 2, 3, 4, 5).map(Integer.valueOf(_)).asJava)
@@ -293,7 +293,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
sql("""ALTER TABLE list_table_area DROP PARTITION(2) WITH DATA""")
val carbonTable2 = CarbonMetadata.getInstance().getCarbonTable("default", "list_table_area")
- val partitionInfo2 = carbonTable2.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo2 = carbonTable2.getPartitionInfo()
val partitionIds2 = partitionInfo2.getPartitionIds
val list_info2 = partitionInfo2.getListInfo
assert(partitionIds2 == List(0, 1, 3, 4, 5).map(Integer.valueOf(_)).asJava)
@@ -312,7 +312,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
sql("""ALTER TABLE range_table_logdate ADD PARTITION ('2017/01/01', '2018/01/01')""")
val carbonTable = CarbonEnv
.getCarbonTable(Option("default"), "range_table_logdate")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
val partitionIds = partitionInfo.getPartitionIds
val range_info = partitionInfo.getRangeInfo
assert(partitionIds == List(0, 1, 2, 3, 4, 5).map(Integer.valueOf(_)).asJava)
@@ -353,7 +353,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
"default",
"range_table_logdate"
)
- val partitionInfo1 = carbonTable1.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo1 = carbonTable1.getPartitionInfo()
val partitionIds1 = partitionInfo1.getPartitionIds
val range_info1 = partitionInfo1.getRangeInfo
assert(partitionIds1 == List(0, 1, 2, 4, 5).map(Integer.valueOf(_)).asJava)
@@ -384,7 +384,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
test("Alter table split partition: List Partition") {
sql("""ALTER TABLE list_table_country SPLIT PARTITION(4) INTO ('Canada', 'Russia', '(Good, NotGood)')""".stripMargin)
val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "list_table_country")
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
val partitionIds = partitionInfo.getPartitionIds
val list_info = partitionInfo.getListInfo
assert(partitionIds == List(0, 1, 2, 3, 6, 7, 8, 5).map(Integer.valueOf(_)).asJava)
@@ -426,7 +426,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
sql("""ALTER TABLE list_table_country DROP PARTITION(8)""")
val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable("default", "list_table_country")
- val partitionInfo1 = carbonTable1.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo1 = carbonTable1.getPartitionInfo()
val partitionIds1 = partitionInfo1.getPartitionIds
val list_info1 = partitionInfo1.getListInfo
assert(partitionIds1 == List(0, 1, 2, 3, 6, 7, 5).map(Integer.valueOf(_)).asJava)
@@ -450,7 +450,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
sql("""ALTER TABLE list_table_country SPLIT PARTITION(9) INTO ('Part4', 'Part2', '(Part1, Part3)')""".stripMargin)
val carbonTable = CarbonEnv
.getCarbonTable(Option("default"), "list_table_country")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
val partitionIds = partitionInfo.getPartitionIds
val list_info = partitionInfo.getListInfo
assert(partitionIds == List(0, 1, 2, 3, 6, 7, 5, 10, 11, 12).map(Integer.valueOf(_)).asJava)
@@ -498,7 +498,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
sql("""ALTER TABLE list_table_area SPLIT PARTITION(6) INTO ('One', '(Two, Three )', 'Four')""".stripMargin)
val carbonTable = CarbonEnv
.getCarbonTable(Option("default"), "list_table_area")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
val partitionIds = partitionInfo.getPartitionIds
val list_info = partitionInfo.getListInfo
assert(partitionIds == List(0, 1, 3, 4, 5, 7, 8, 9).map(Integer.valueOf(_)).asJava)
@@ -542,7 +542,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
sql("""ALTER TABLE range_table_logdate_split SPLIT PARTITION(4) INTO ('2017/01/01', '2018/01/01')""")
val carbonTable = CarbonEnv
.getCarbonTable(Option("default"), "range_table_logdate_split")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
val partitionIds = partitionInfo.getPartitionIds
val rangeInfo = partitionInfo.getRangeInfo
assert(partitionIds == List(0, 1, 2, 3, 5, 6).map(Integer.valueOf(_)).asJava)
@@ -583,7 +583,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
"default",
"range_table_logdate_split"
)
- val partitionInfo1 = carbonTable1.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo1 = carbonTable1.getPartitionInfo()
val partitionIds1 = partitionInfo1.getPartitionIds
val rangeInfo1 = partitionInfo1.getRangeInfo
assert(partitionIds1 == List(0, 1, 2, 3, 5).map(Integer.valueOf(_)).asJava)
@@ -604,7 +604,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
sql("""ALTER TABLE range_table_bucket SPLIT PARTITION(4) INTO ('2017/01/01', '2018/01/01')""")
val carbonTable = CarbonEnv
.getCarbonTable(Option("default"), "range_table_bucket")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
val partitionIds = partitionInfo.getPartitionIds
val rangeInfo = partitionInfo.getRangeInfo
assert(partitionIds == List(0, 1, 2, 3, 5, 6).map(Integer.valueOf(_)).asJava)
@@ -642,7 +642,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
sql("""ALTER TABLE range_table_bucket DROP PARTITION(6) WITH DATA""")
val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable("default", "range_table_bucket")
- val partitionInfo1 = carbonTable1.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo1 = carbonTable1.getPartitionInfo()
val partitionIds1 = partitionInfo1.getPartitionIds
val rangeInfo1 = partitionInfo1.getRangeInfo
assert(partitionIds1 == List(0, 1, 2, 3, 5).map(Integer.valueOf(_)).asJava)
@@ -660,7 +660,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
sql("""ALTER TABLE range_table_bucket DROP PARTITION(3)""")
val carbonTable2 = CarbonMetadata.getInstance().getCarbonTable("default", "range_table_bucket")
- val partitionInfo2 = carbonTable2.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo2 = carbonTable2.getPartitionInfo()
val partitionIds2 = partitionInfo2.getPartitionIds
val rangeInfo2 = partitionInfo2.getRangeInfo
assert(partitionIds2 == List(0, 1, 2, 5).map(Integer.valueOf(_)).asJava)
@@ -677,7 +677,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
sql("""ALTER TABLE range_table_bucket DROP PARTITION(5)""")
val carbonTable3 = CarbonMetadata.getInstance().getCarbonTable("default", "range_table_bucket")
- val partitionInfo3 = carbonTable3.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo3 = carbonTable3.getPartitionInfo()
val partitionIds3 = partitionInfo3.getPartitionIds
val rangeInfo3 = partitionInfo3.getRangeInfo
assert(partitionIds3 == List(0, 1, 2).map(Integer.valueOf(_)).asJava)
@@ -807,7 +807,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
sql("ALTER TABLE carbon_table_default_db ADD PARTITION ('2017')")
val carbonTable = CarbonEnv
.getCarbonTable(Option("default"), "carbon_table_default_db")(sqlContext.sparkSession)
- val partitionInfo = carbonTable.getPartitionInfo(carbonTable.getTableName)
+ val partitionInfo = carbonTable.getPartitionInfo()
val partitionIds = partitionInfo.getPartitionIds
val range_info = partitionInfo.getRangeInfo
assert(partitionIds == List(0, 1, 2, 3).map(Integer.valueOf(_)).asJava)
@@ -827,7 +827,7 @@ class TestAlterPartitionTable extends QueryTest with BeforeAndAfterAll {
sql("ALTER TABLE carbondb.carbontable ADD PARTITION ('2017')")
val carbonTable1 = CarbonMetadata.getInstance().getCarbonTable("carbondb", "carbontable")
- val partitionInfo1 = carbonTable1.getPartitionInfo(carbonTable1.getTableName)
+ val partitionInfo1 = carbonTable1.getPartitionInfo()
val partitionIds1 = partitionInfo1.getPartitionIds
val range_info1 = partitionInfo1.getRangeInfo
assert(partitionIds1 == List(0, 1, 2, 3).map(Integer.valueOf(_)).asJava)
diff --git a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
index a465251..5109ad2 100644
--- a/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
+++ b/integration/spark2/src/test/scala/org/apache/carbondata/spark/util/DictionaryTestCaseUtil.scala
@@ -39,7 +39,7 @@ object DictionaryTestCaseUtil {
*/
def checkDictionary(relation: CarbonRelation, columnName: String, value: String) {
val table = relation.carbonTable
- val dimension = table.getDimensionByName(table.getTableName, columnName)
+ val dimension = table.getDimensionByName(columnName)
val tableIdentifier = new CarbonTableIdentifier(table.getDatabaseName, table.getTableName, "uniqueid")
val absoluteTableIdentifier = AbsoluteTableIdentifier.from(table.getTablePath, tableIdentifier)
val columnIdentifier = new DictionaryColumnUniqueIdentifier(absoluteTableIdentifier,
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
index 8aa0cf6..a46c472 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
@@ -54,7 +54,7 @@ class TableBucketingTestCase extends Spark2QueryTest with BeforeAndAfterAll {
"('BUCKETNUMBER'='4', 'BUCKETCOLUMNS'='name')")
sql(s"LOAD DATA INPATH '$resourcesPath/source.csv' INTO TABLE t4")
val table = CarbonEnv.getCarbonTable(Option("default"), "t4")(sqlContext.sparkSession)
- if (table != null && table.getBucketingInfo("t4") != null) {
+ if (table != null && table.getBucketingInfo() != null) {
assert(true)
} else {
assert(false, "Bucketing info does not exist")
@@ -69,7 +69,7 @@ class TableBucketingTestCase extends Spark2QueryTest with BeforeAndAfterAll {
sql(s"LOAD DATA INPATH '$resourcesPath/source.csv' INTO TABLE t10")
CarbonProperties.getInstance().addProperty(CarbonCommonConstants.ENABLE_UNSAFE_SORT, "false")
val table: CarbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "t10")
- if (table != null && table.getBucketingInfo("t10") != null) {
+ if (table != null && table.getBucketingInfo() != null) {
assert(true)
} else {
assert(false, "Bucketing info does not exist")
diff --git a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala
index d368a8e..e36e814 100644
--- a/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala
+++ b/integration/spark2/src/test/scala/org/apache/spark/carbondata/restructure/vectorreader/AlterTableColumnRenameTestCase.scala
@@ -34,8 +34,8 @@ class AlterTableColumnRenameTestCase extends Spark2QueryTest with BeforeAndAfter
test("test only column rename operation") {
sql("alter table rename change empname empAddress string")
val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "rename")
- assert(null != carbonTable.getColumnByName("rename", "empAddress"))
- assert(null == carbonTable.getColumnByName("rename", "empname"))
+ assert(null != carbonTable.getColumnByName("empAddress"))
+ assert(null == carbonTable.getColumnByName("empname"))
}
test("test only column rename operation with datatype change also") {
@@ -46,8 +46,8 @@ class AlterTableColumnRenameTestCase extends Spark2QueryTest with BeforeAndAfter
}
sql("alter table rename change deptno classNo Bigint")
val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "rename")
- assert(null != carbonTable.getColumnByName("rename", "classNo"))
- assert(null == carbonTable.getColumnByName("rename", "deptno"))
+ assert(null != carbonTable.getColumnByName("classNo"))
+ assert(null == carbonTable.getColumnByName("deptno"))
}
test("test trying to rename column which does not exists") {
@@ -74,10 +74,10 @@ class AlterTableColumnRenameTestCase extends Spark2QueryTest with BeforeAndAfter
sql("alter table rename change projectenddate newDate Timestamp")
sql("alter table rename change workgroupcategory newCategory int")
val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "rename")
- assert(null != carbonTable.getColumnByName("rename", "newDate"))
- assert(null == carbonTable.getColumnByName("rename", "projectenddate"))
- assert(null != carbonTable.getColumnByName("rename", "newCategory"))
- assert(null == carbonTable.getColumnByName("rename", "workgroupcategory"))
+ assert(null != carbonTable.getColumnByName("newDate"))
+ assert(null == carbonTable.getColumnByName("projectenddate"))
+ assert(null != carbonTable.getColumnByName("newCategory"))
+ assert(null == carbonTable.getColumnByName("workgroupcategory"))
}
test("query count after column rename and filter results"){
@@ -114,14 +114,14 @@ class AlterTableColumnRenameTestCase extends Spark2QueryTest with BeforeAndAfter
createTableAndLoad()
sql("alter table rename add columns(newAdded string)")
var carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "rename")
- assert(null != carbonTable.getColumnByName("rename", "newAdded"))
+ assert(null != carbonTable.getColumnByName("newAdded"))
sql("alter table rename change newAdded addedRename string")
carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "rename")
- assert(null != carbonTable.getColumnByName("rename", "addedRename"))
- assert(null == carbonTable.getColumnByName("rename", "newAdded"))
+ assert(null != carbonTable.getColumnByName("addedRename"))
+ assert(null == carbonTable.getColumnByName("newAdded"))
sql("alter table rename drop columns(addedRename)")
carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "rename")
- assert(null == carbonTable.getColumnByName("rename", "addedRename"))
+ assert(null == carbonTable.getColumnByName("addedRename"))
intercept[ProcessMetaDataException] {
sql("alter table rename change addedRename test string")
}
@@ -303,8 +303,8 @@ class AlterTableColumnRenameTestCase extends Spark2QueryTest with BeforeAndAfter
sql("create table biginttable(name string, age int, bigintfield bigint) stored by 'carbondata'")
sql("alter table biginttable change bigintfield testfield bigint")
val carbonTable = CarbonMetadata.getInstance().getCarbonTable("default", "biginttable")
- assert(null != carbonTable.getColumnByName("biginttable", "testfield"))
- assert(null == carbonTable.getColumnByName("biginttable", "bigintfield"))
+ assert(null != carbonTable.getColumnByName("testfield"))
+ assert(null == carbonTable.getColumnByName("bigintfield"))
sql("drop table if exists biginttable")
}
diff --git a/log b/log
new file mode 100644
index 0000000..e69de29
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
index 233b4cd..bee98f6 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/DataLoadProcessBuilder.java
@@ -262,10 +262,8 @@ public final class DataLoadProcessBuilder {
configuration.setDataLoadProperty(CarbonLoadOptionConstants.CARBON_OPTIONS_BINARY_DECODER,
loadModel.getBinaryDecoder());
- List<CarbonDimension> dimensions =
- carbonTable.getDimensionByTableName(carbonTable.getTableName());
- List<CarbonMeasure> measures =
- carbonTable.getMeasureByTableName(carbonTable.getTableName());
+ List<CarbonDimension> dimensions = carbonTable.getVisibleDimensions();
+ List<CarbonMeasure> measures = carbonTable.getVisibleMeasures();
List<DataField> dataFields = new ArrayList<>();
List<DataField> complexDataFields = new ArrayList<>();
@@ -304,7 +302,7 @@ public final class DataLoadProcessBuilder {
}
configuration.setDataFields(
updateDataFieldsBasedOnSortColumns(dataFields).toArray(new DataField[dataFields.size()]));
- configuration.setBucketingInfo(carbonTable.getBucketingInfo(carbonTable.getTableName()));
+ configuration.setBucketingInfo(carbonTable.getBucketingInfo());
// configuration for one pass load: dictionary server info
configuration.setUseOnePass(loadModel.getUseOnePass());
configuration.setDictionaryServerHost(loadModel.getDictionaryServerHost());
@@ -341,7 +339,7 @@ public final class DataLoadProcessBuilder {
*/
private static void setSortColumnInfo(CarbonTable carbonTable, CarbonLoadModel loadModel,
CarbonDataLoadConfiguration configuration) {
- List<String> sortCols = carbonTable.getSortColumns(carbonTable.getTableName());
+ List<String> sortCols = carbonTable.getSortColumns();
SortScopeOptions.SortScope sortScope = SortScopeOptions.getSortScope(loadModel.getSortScope());
if (!SortScopeOptions.SortScope.LOCAL_SORT.equals(sortScope)
|| sortCols.size() == 0
diff --git a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
index b1a3a98..8d5ee0e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/loading/model/CarbonLoadModelBuilder.java
@@ -73,7 +73,7 @@ public class CarbonLoadModelBuilder {
Map<String, String> optionsFinal = LoadOption.fillOptionWithDefaultValue(options);
if (!options.containsKey("fileheader")) {
- List<CarbonColumn> csvHeader = table.getCreateOrderColumn(table.getTableName());
+ List<CarbonColumn> csvHeader = table.getCreateOrderColumn();
String[] columns = new String[csvHeader.size()];
for (int i = 0; i < columns.length; i++) {
columns[i] = csvHeader.get(i).getColName();
@@ -213,7 +213,7 @@ public class CarbonLoadModelBuilder {
}
} else {
if (StringUtils.isEmpty(fileHeader)) {
- List<CarbonColumn> columns = table.getCreateOrderColumn(table.getTableName());
+ List<CarbonColumn> columns = table.getCreateOrderColumn();
List<String> columnNames = new ArrayList<>();
List<String> partitionColumns = new ArrayList<>();
for (int i = 0; i < columns.size(); i++) {
@@ -395,7 +395,7 @@ public class CarbonLoadModelBuilder {
if (sortScope != null) {
// We support global sort for Hive standard partition, but don't support
// global sort for other partition type.
- if (table.getPartitionInfo(table.getTableName()) != null &&
+ if (table.getPartitionInfo() != null &&
!table.isHivePartitionTable() &&
sortScope.equalsIgnoreCase(SortScopeOptions.SortScope.GLOBAL_SORT.toString())) {
throw new InvalidLoadOptionException("Don't support use global sort on "
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
index bc65b75..d42b46a 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CarbonCompactionUtil.java
@@ -362,8 +362,7 @@ public class CarbonCompactionUtil {
*/
public static int[] updateColumnSchemaAndGetCardinality(Map<String, Integer> columnCardinalityMap,
CarbonTable carbonTable, List<ColumnSchema> updatedColumnSchemaList) {
- List<CarbonDimension> masterDimensions =
- carbonTable.getDimensionByTableName(carbonTable.getTableName());
+ List<CarbonDimension> masterDimensions = carbonTable.getVisibleDimensions();
List<Integer> updatedCardinalityList = new ArrayList<>(columnCardinalityMap.size());
for (CarbonDimension dimension : masterDimensions) {
Integer value = columnCardinalityMap.get(dimension.getColumnId());
@@ -380,8 +379,7 @@ public class CarbonCompactionUtil {
}
}
// add measures to the column schema list
- List<CarbonMeasure> masterSchemaMeasures =
- carbonTable.getMeasureByTableName(carbonTable.getTableName());
+ List<CarbonMeasure> masterSchemaMeasures = carbonTable.getVisibleMeasures();
for (CarbonMeasure measure : masterSchemaMeasures) {
updatedColumnSchemaList.add(measure.getColumnSchema());
}
@@ -634,7 +632,7 @@ public class CarbonCompactionUtil {
return false;
}
List<CarbonDimension> sortColumnsOfTable = new ArrayList<>();
- for (CarbonDimension dimension : table.getDimensions()) {
+ for (CarbonDimension dimension : table.getVisibleDimensions()) {
if (dimension.isSortColumn()) {
sortColumnsOfTable.add(dimension);
}
diff --git a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
index a64a183..d9a84fe 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/merger/CompactionResultSortProcessor.java
@@ -431,7 +431,7 @@ public class CompactionResultSortProcessor extends AbstractResultProcessor {
* create an instance of sort data rows
*/
private void initSortDataRows() throws Exception {
- measureCount = carbonTable.getMeasureByTableName(tableName).size();
+ measureCount = carbonTable.getVisibleMeasures().size();
dimensions = new ArrayList<>(2);
dimensions.addAll(segmentProperties.getDimensions());
dimensions.addAll(segmentProperties.getComplexDimensions());
diff --git a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java
index 8bb03cd..6a5dbdc 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/sort/sortdata/TableFieldStat.java
@@ -121,8 +121,7 @@ public class TableFieldStat implements Serializable {
int tmpVarcharCnt = 0;
int tmpComplexcount = 0;
- List<CarbonDimension> allDimensions =
- sortParameters.getCarbonTable().getDimensionByTableName(sortParameters.getTableName());
+ List<CarbonDimension> allDimensions = sortParameters.getCarbonTable().getVisibleDimensions();
List<CarbonDimension> updatedDimensions = updateDimensionsBasedOnSortColumns(allDimensions);
for (int i = 0; i < updatedDimensions.size(); i++) {
CarbonDimension carbonDimension = updatedDimensions.get(i);
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 ace8cc3..c0d05d1 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
@@ -210,8 +210,7 @@ public class CarbonFactDataHandlerModel {
CarbonTable carbonTable = configuration.getTableSpec().getCarbonTable();
List<ColumnSchema> wrapperColumnSchema = CarbonUtil
- .getColumnSchemaList(carbonTable.getDimensionByTableName(identifier.getTableName()),
- carbonTable.getMeasureByTableName(identifier.getTableName()));
+ .getColumnSchemaList(carbonTable.getVisibleDimensions(), carbonTable.getVisibleMeasures());
int[] colCardinality =
CarbonUtil.getFormattedCardinality(dimLensWithComplex, wrapperColumnSchema);
@@ -326,7 +325,7 @@ public class CarbonFactDataHandlerModel {
String[] tempStoreLocation, String carbonDataDirectoryPath) {
// for dynamic page size in write step if varchar columns exist
- List<CarbonDimension> allDimensions = carbonTable.getDimensions();
+ List<CarbonDimension> allDimensions = carbonTable.getVisibleDimensions();
CarbonColumn[] noDicAndComplexColumns =
new CarbonColumn[segmentProperties.getNumberOfNoDictionaryDimension() + segmentProperties
.getComplexDimensions().size()];
@@ -354,8 +353,7 @@ public class CarbonFactDataHandlerModel {
segmentProperties.getDimensions().size() - carbonFactDataHandlerModel
.getNoDictionaryCount());
List<ColumnSchema> wrapperColumnSchema = CarbonUtil
- .getColumnSchemaList(carbonTable.getDimensionByTableName(tableName),
- carbonTable.getMeasureByTableName(tableName));
+ .getColumnSchemaList(carbonTable.getVisibleDimensions(), carbonTable.getVisibleMeasures());
carbonFactDataHandlerModel.setWrapperColumnSchema(wrapperColumnSchema);
// get the cardinality for all all the columns including no
// dictionary columns and complex columns
diff --git a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
index 1cdcad5..e3ed55e 100644
--- a/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
+++ b/processing/src/main/java/org/apache/carbondata/processing/util/CarbonDataProcessorUtil.java
@@ -376,13 +376,12 @@ public final class CarbonDataProcessorUtil {
*/
public static Set<String> getSchemaColumnNames(CarbonDataLoadSchema schema) {
Set<String> columnNames = new HashSet<String>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
- String factTableName = schema.getCarbonTable().getTableName();
List<CarbonDimension> dimensions =
- schema.getCarbonTable().getDimensionByTableName(factTableName);
+ schema.getCarbonTable().getVisibleDimensions();
for (CarbonDimension dimension : dimensions) {
columnNames.add(dimension.getColName());
}
- List<CarbonMeasure> measures = schema.getCarbonTable().getMeasureByTableName(factTableName);
+ List<CarbonMeasure> measures = schema.getCarbonTable().getVisibleMeasures();
for (CarbonMeasure msr : measures) {
columnNames.add(msr.getColName());
}
@@ -394,7 +393,7 @@ public final class CarbonDataProcessorUtil {
for (int i = 0; i < type.length; i++) {
type[i] = DataTypes.DOUBLE;
}
- List<CarbonMeasure> measures = carbonTable.getMeasureByTableName(carbonTable.getTableName());
+ List<CarbonMeasure> measures = carbonTable.getVisibleMeasures();
for (int i = 0; i < type.length; i++) {
type[i] = measures.get(i).getDataType();
}
@@ -408,8 +407,7 @@ public final class CarbonDataProcessorUtil {
* @return
*/
public static DataType[] getNoDictDataTypes(CarbonTable carbonTable) {
- List<CarbonDimension> dimensions =
- carbonTable.getDimensionByTableName(carbonTable.getTableName());
+ List<CarbonDimension> dimensions = carbonTable.getVisibleDimensions();
List<DataType> type = new ArrayList<>();
for (int i = 0; i < dimensions.size(); i++) {
if (dimensions.get(i).isSortColumn() && !dimensions.get(i).hasEncoding(Encoding.DICTIONARY)) {
@@ -426,8 +424,7 @@ public final class CarbonDataProcessorUtil {
* @return
*/
public static boolean[] getNoDictSortColMapping(CarbonTable carbonTable) {
- List<CarbonDimension> dimensions =
- carbonTable.getDimensionByTableName(carbonTable.getTableName());
+ List<CarbonDimension> dimensions = carbonTable.getVisibleDimensions();
List<Boolean> noDicSortColMap = new ArrayList<>();
for (int i = 0; i < dimensions.size(); i++) {
if (dimensions.get(i).isSortColumn()) {
@@ -455,8 +452,7 @@ public final class CarbonDataProcessorUtil {
* data.
*/
public static int[] getColumnIdxBasedOnSchemaInRow(CarbonTable carbonTable) {
- List<CarbonDimension> dimensions =
- carbonTable.getDimensionByTableName(carbonTable.getTableName());
+ List<CarbonDimension> dimensions = carbonTable.getVisibleDimensions();
List<Integer> noDicSortColMap = new ArrayList<>();
int counter = 0;
for (CarbonDimension dimension : dimensions) {
@@ -483,8 +479,7 @@ public final class CarbonDataProcessorUtil {
* @return
*/
public static Map<String, DataType[]> getNoDictSortAndNoSortDataTypes(CarbonTable carbonTable) {
- List<CarbonDimension> dimensions =
- carbonTable.getDimensionByTableName(carbonTable.getTableName());
+ List<CarbonDimension> dimensions = carbonTable.getVisibleDimensions();
List<DataType> noDictSortType = new ArrayList<>();
List<DataType> noDictNoSortType = new ArrayList<>();
for (int i = 0; i < dimensions.size(); i++) {
@@ -522,7 +517,7 @@ public final class CarbonDataProcessorUtil {
for (int i = 0; i < type.length; i++) {
type[i] = DataTypes.DOUBLE;
}
- List<CarbonMeasure> measures = carbonTable.getMeasureByTableName(tableName);
+ List<CarbonMeasure> measures = carbonTable.getVisibleMeasures();
for (int i = 0; i < measureCount; i++) {
type[i] = measures.get(i).getDataType();
}
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
index 3a92bc1..ffad6b1 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
@@ -573,7 +573,7 @@ public class CarbonWriterBuilder {
carbonTable.getTableInfo().setTablePath(path);
carbonTable.setTransactionalTable(false);
List<ColumnSchema> columnSchemas =
- carbonTable.getCreateOrderColumn(carbonTable.getTableName()).stream().map(
+ carbonTable.getCreateOrderColumn().stream().map(
CarbonColumn::getColumnSchema
).collect(Collectors.toList());
schema = new Schema(columnSchemas);
diff --git a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordWriter.java b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordWriter.java
index e9b9e6e..1363e13 100644
--- a/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordWriter.java
+++ b/streaming/src/main/java/org/apache/carbondata/streaming/CarbonStreamRecordWriter.java
@@ -311,8 +311,7 @@ public class CarbonStreamRecordWriter extends RecordWriter<Void, Object> {
private void writeFileHeader() throws IOException {
List<ColumnSchema> wrapperColumnSchemaList = CarbonUtil
- .getColumnSchemaList(carbonTable.getDimensionByTableName(carbonTable.getTableName()),
- carbonTable.getMeasureByTableName(carbonTable.getTableName()));
+ .getColumnSchemaList(carbonTable.getVisibleDimensions(), carbonTable.getVisibleMeasures());
int[] dimLensWithComplex = new int[wrapperColumnSchemaList.size()];
for (int i = 0; i < dimLensWithComplex.length; i++) {
dimLensWithComplex[i] = Integer.MAX_VALUE;