carbondata-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From jack...@apache.org
Subject [4/5] incubator-carbondata git commit: change cube to table
Date Tue, 19 Jul 2016 08:38:52 GMT
change cube to table

change Cube to Table

fix style

change schemaName to databaseName

change SchemaName to DatabaseName


Project: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/commit/84b476bc
Tree: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/tree/84b476bc
Diff: http://git-wip-us.apache.org/repos/asf/incubator-carbondata/diff/84b476bc

Branch: refs/heads/master
Commit: 84b476bc41b77d9736da916949e105dda28df31c
Parents: 6948cb0
Author: jackylk <jacky.likun@huawei.com>
Authored: Sun Jul 17 22:57:58 2016 +0800
Committer: jackylk <jacky.likun@huawei.com>
Committed: Tue Jul 19 16:33:39 2016 +0800

----------------------------------------------------------------------
 .../common/logging/impl/StandardLogService.java |   4 +-
 .../common/ext/ColumnUniqueIdGenerator.java     |   2 +-
 .../carbondata/core/cache/CarbonLRUCache.java   |   2 +-
 .../core/carbon/CarbonDataLoadSchema.java       |   2 +-
 .../carbon/metadata/schema/table/TableInfo.java |   2 +-
 .../core/service/ColumnUniqueIdService.java     |   4 +-
 .../org/carbondata/core/util/CarbonUtil.java    |  34 +--
 .../org/carbondata/scan/filter/FilterUtil.java  |   2 +-
 .../carbondata/scan/model/CarbonQueryPlan.java  |  44 ++--
 .../examples/GenerateDictionaryExample.scala    |   2 +-
 .../hadoop/test/util/StoreCreator.java          |  46 ++--
 .../spark/merger/CarbonCompactionExecutor.java  |   8 +-
 .../spark/merger/CompactionCallable.java        |   8 +-
 .../spark/merger/RowResultMerger.java           |  20 +-
 .../carbondata/spark/load/CarbonLoaderUtil.java |  79 ++++---
 .../partition/api/impl/CSVFilePartitioner.java  |  23 +-
 .../api/impl/QueryPartitionHelper.java          |  44 ++--
 .../carbondata/spark/util/CarbonQueryUtil.java  |   6 +-
 .../carbondata/spark/util/LoadMetadataUtil.java |   6 +-
 .../spark/sql/CarbonCatalystOperators.scala     |   4 +-
 .../org/apache/spark/sql/CarbonContext.scala    |  16 +-
 .../spark/sql/CarbonDatasourceRelation.scala    |  28 ++-
 .../org/apache/spark/sql/CarbonOperators.scala  |  10 +-
 .../org/apache/spark/sql/CarbonSqlParser.scala  |  29 +--
 .../execution/command/carbonTableSchema.scala   | 228 ++++++++++---------
 .../spark/sql/hive/CarbonMetastoreCatalog.scala | 172 +++++++-------
 .../spark/sql/hive/CarbonStrategies.scala       |  14 +-
 .../spark/rdd/CarbonCleanFilesRDD.scala         |   6 +-
 .../spark/rdd/CarbonDataLoadRDD.scala           |  14 +-
 .../spark/rdd/CarbonDataPartitionRDD.scala      |   6 +-
 .../spark/rdd/CarbonDataRDDFactory.scala        |  81 ++++---
 .../spark/rdd/CarbonDeleteLoadByDateRDD.scala   |   6 +-
 .../spark/rdd/CarbonDeleteLoadRDD.scala         |   6 +-
 .../spark/rdd/CarbonDropTableRDD.scala          |   6 +-
 .../carbondata/spark/rdd/CarbonMergerRDD.scala  |  10 +-
 .../carbondata/spark/rdd/CarbonScanRDD.scala    |   2 +-
 .../org/carbondata/spark/rdd/Compactor.scala    |   8 +-
 .../spark/util/GlobalDictionaryUtil.scala       |   2 +-
 .../AllDataTypesTestCaseAggregate.scala         |  18 +-
 .../aggquery/IntegerDataTypeTestCase.scala      |  10 +-
 .../DataCompactionCardinalityBoundryTest.scala  |   2 +-
 .../datacompaction/DataCompactionLockTest.scala |   2 +-
 .../datacompaction/DataCompactionTest.scala     |   2 +-
 .../dataload/TestLoadDataWithHiveSyntax.scala   |  23 +-
 .../deleteTable/TestDeleteTableNewDDL.scala     |   5 +-
 .../detailquery/AllDataTypesTestCase.scala      |  10 +-
 .../HighCardinalityDataTypesTestCase.scala      |  72 +++---
 .../detailquery/IntegerDataTypeTestCase.scala   |  10 +-
 ...estampDataTypeDirectDictionaryTestCase.scala |  29 ++-
 .../filterexpr/AllDataTypesTestCaseFilter.scala |  10 +-
 .../filterexpr/IntegerDataTypeTestCase.scala    |  10 +-
 .../joinquery/AllDataTypesTestCaseJoin.scala    |  10 +-
 .../joinquery/IntegerDataTypeTestCase.scala     |  10 +-
 .../sortexpr/AllDataTypesTestCaseSort.scala     |  10 +-
 .../sortexpr/IntegerDataTypeTestCase.scala      |  10 +-
 .../AutoHighCardinalityIdentifyTestCase.scala   |  12 +-
 .../util/GlobalDictionaryUtilTestCase.scala     |  22 +-
 .../org/carbondata/lcm/locks/LocalFileLock.java |  26 +--
 .../carbondata/lcm/locks/ZooKeeperLocking.java  |  14 +-
 .../lcm/status/SegmentStatusManager.java        |  14 +-
 .../processing/api/dataloader/SchemaInfo.java   |  28 +--
 .../processing/csvload/DataGraphExecuter.java   |   2 +-
 .../processing/csvload/GraphExecutionUtil.java  |   2 +-
 .../dataprocessor/DataProcessTaskStatus.java    |  39 +---
 .../dataprocessor/IDataProcessStatus.java       |  20 +-
 .../graphgenerator/GraphGenerator.java          |  42 ++--
 .../configuration/GraphConfigurationInfo.java   |  14 +-
 .../processing/mdkeygen/MDKeyGenStep.java       |  10 +-
 .../processing/mdkeygen/MDKeyGenStepMeta.java   |  55 ++---
 .../merger/step/CarbonSliceMergerStep.java      |   5 +-
 .../merger/step/CarbonSliceMergerStepMeta.java  |  52 ++---
 .../processing/schema/metadata/ColumnsInfo.java |  43 +---
 .../sortandgroupby/sortdata/SortDataRows.java   |  20 +-
 .../sortdatastep/SortKeyStep.java               |   2 +-
 .../sortdatastep/SortKeyStepMeta.java           |  52 ++---
 .../csvbased/BadRecordslogger.java              |   4 +-
 .../csvbased/CarbonCSVBasedSeqGenMeta.java      |  41 ++--
 .../csvbased/CarbonCSVBasedSeqGenStep.java      |  19 +-
 .../FileStoreSurrogateKeyGenForCSV.java         |   6 +-
 .../processing/util/CarbonSchemaParser.java     |  40 ++--
 80 files changed, 857 insertions(+), 956 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/common/src/main/java/org/carbondata/common/logging/impl/StandardLogService.java
----------------------------------------------------------------------
diff --git a/common/src/main/java/org/carbondata/common/logging/impl/StandardLogService.java b/common/src/main/java/org/carbondata/common/logging/impl/StandardLogService.java
index 07cd7bb..2592e2a 100644
--- a/common/src/main/java/org/carbondata/common/logging/impl/StandardLogService.java
+++ b/common/src/main/java/org/carbondata/common/logging/impl/StandardLogService.java
@@ -125,8 +125,8 @@ public final class StandardLogService implements LogService {
     StandardLogService.doLog = doLog;
   }
 
-  public static String getPartitionID(String cubeName) {
-    return cubeName.substring(cubeName.lastIndexOf('_') + 1, cubeName.length());
+  public static String getPartitionID(String tableName) {
+    return tableName.substring(tableName.lastIndexOf('_') + 1, tableName.length());
   }
 
   public static void setThreadName(String partitionID, String queryID) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/core/src/main/java/org/carbondata/common/ext/ColumnUniqueIdGenerator.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/common/ext/ColumnUniqueIdGenerator.java b/core/src/main/java/org/carbondata/common/ext/ColumnUniqueIdGenerator.java
index 0ef3950..bc96302 100644
--- a/core/src/main/java/org/carbondata/common/ext/ColumnUniqueIdGenerator.java
+++ b/core/src/main/java/org/carbondata/common/ext/ColumnUniqueIdGenerator.java
@@ -30,7 +30,7 @@ public class ColumnUniqueIdGenerator implements ColumnUniqueIdService {
 
   private static ColumnUniqueIdService columnUniqueIdService = new ColumnUniqueIdGenerator();
 
-  @Override public String generateUniqueId(String schemaName, ColumnSchema columnSchema) {
+  @Override public String generateUniqueId(String databaseName, ColumnSchema columnSchema) {
     return UUID.randomUUID().toString();
   }
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/core/src/main/java/org/carbondata/core/cache/CarbonLRUCache.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/cache/CarbonLRUCache.java b/core/src/main/java/org/carbondata/core/cache/CarbonLRUCache.java
index 71259a9..72ee209 100644
--- a/core/src/main/java/org/carbondata/core/cache/CarbonLRUCache.java
+++ b/core/src/main/java/org/carbondata/core/cache/CarbonLRUCache.java
@@ -44,7 +44,7 @@ public final class CarbonLRUCache {
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(CarbonLRUCache.class.getName());
   /**
-   * Map that will contain key as cube unique name and value as cache Holder
+   * Map that will contain key as table unique name and value as cache Holder
    * object
    */
   private Map<String, Cacheable> lruCacheMap;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/core/src/main/java/org/carbondata/core/carbon/CarbonDataLoadSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/CarbonDataLoadSchema.java b/core/src/main/java/org/carbondata/core/carbon/CarbonDataLoadSchema.java
index 31e8243..f23f600 100644
--- a/core/src/main/java/org/carbondata/core/carbon/CarbonDataLoadSchema.java
+++ b/core/src/main/java/org/carbondata/core/carbon/CarbonDataLoadSchema.java
@@ -84,7 +84,7 @@ public class CarbonDataLoadSchema implements Serializable {
 
   /**
    * Dimension Relation object which will be filled from
-   * Load DML Command to support normalized cube data load
+   * Load DML Command to support normalized table data load
    */
   public static class DimensionRelation implements Serializable {
     /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/TableInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/TableInfo.java b/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/TableInfo.java
index 0b3348c..0a9cfca 100644
--- a/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/TableInfo.java
+++ b/core/src/main/java/org/carbondata/core/carbon/metadata/schema/table/TableInfo.java
@@ -56,7 +56,7 @@ public class TableInfo implements Serializable {
   private List<TableSchema> aggregateTableList;
 
   /**
-   * last updated time to update the cube if any changes
+   * last updated time to update the table if any changes
    */
   private long lastUpdatedTime;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/core/src/main/java/org/carbondata/core/service/ColumnUniqueIdService.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/service/ColumnUniqueIdService.java b/core/src/main/java/org/carbondata/core/service/ColumnUniqueIdService.java
index 9c65e3f..e2e0807 100644
--- a/core/src/main/java/org/carbondata/core/service/ColumnUniqueIdService.java
+++ b/core/src/main/java/org/carbondata/core/service/ColumnUniqueIdService.java
@@ -26,9 +26,9 @@ import org.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
 public interface ColumnUniqueIdService {
 
   /**
-   * @param schemaName
+   * @param databaseName
    * @param columnSchema
    * @return generate unique id
    */
-  public String generateUniqueId(String schemaName, ColumnSchema columnSchema);
+  public String generateUniqueId(String databaseName, ColumnSchema columnSchema);
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
index 2368139..c63dccc 100644
--- a/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
+++ b/core/src/main/java/org/carbondata/core/util/CarbonUtil.java
@@ -458,17 +458,17 @@ public final class CarbonUtil {
   }
 
   /**
-   * This function will rename the cube to be deleted
+   * This function will rename the table to be deleted
    *
    * @param partitionCount
    * @param storePath
-   * @param schemaName
-   * @param cubeName
+   * @param databaseName
+   * @param tableName
    */
-  public static void renameCubeForDeletion(int partitionCount, String storePath, String schemaName,
-      String cubeName) {
-    String cubeNameWithPartition = "";
-    String schemaNameWithPartition = "";
+  public static void renameTableForDeletion(int partitionCount, String storePath,
+      String databaseName, String tableName) {
+    String tableNameWithPartition = "";
+    String databaseNameWithPartition = "";
     String fullPath = "";
     String newFilePath = "";
     String newFileName = "";
@@ -477,20 +477,20 @@ public final class CarbonUtil {
     FileFactory.FileType fileType = null;
     ExecutorService executorService = Executors.newFixedThreadPool(10);
     for (int i = 0; i < partitionCount; i++) {
-      schemaNameWithPartition = schemaName + '_' + i;
-      cubeNameWithPartition = cubeName + '_' + i;
-      newFileName = cubeNameWithPartition + '_' + time;
-      fullPath = storePath + File.separator + schemaNameWithPartition + File.separator
-          + cubeNameWithPartition;
+      databaseNameWithPartition = databaseName + '_' + i;
+      tableNameWithPartition = tableName + '_' + i;
+      newFileName = tableNameWithPartition + '_' + time;
+      fullPath = storePath + File.separator + databaseNameWithPartition + File.separator
+          + tableNameWithPartition;
       newFilePath =
-          storePath + File.separator + schemaNameWithPartition + File.separator + newFileName;
+          storePath + File.separator + databaseNameWithPartition + File.separator + newFileName;
       fileType = FileFactory.getFileType(fullPath);
       try {
         if (FileFactory.isFileExist(fullPath, fileType)) {
           CarbonFile file = FileFactory.getCarbonFile(fullPath, fileType);
           boolean isRenameSuccessfull = file.renameTo(newFilePath);
           if (!isRenameSuccessfull) {
-            LOGGER.error("Problem renaming the cube :: " + fullPath);
+            LOGGER.error("Problem renaming the table :: " + fullPath);
             c = new DeleteFolderAndFiles(file);
             executorService.submit(c);
           } else {
@@ -499,7 +499,7 @@ public final class CarbonUtil {
           }
         }
       } catch (IOException e) {
-        LOGGER.error("Problem renaming the cube :: " + fullPath);
+        LOGGER.error("Problem renaming the table :: " + fullPath);
       }
     }
     executorService.shutdown();
@@ -867,7 +867,7 @@ public final class CarbonUtil {
     }
   }
 
-  public static String getCarbonStorePath(String schemaName, String cubeName) {
+  public static String getCarbonStorePath(String databaseName, String tableName) {
     CarbonProperties prop = CarbonProperties.getInstance();
     if (null == prop) {
       return null;
@@ -1075,7 +1075,7 @@ public final class CarbonUtil {
   }
 
   /**
-   * Thread to delete the cubes
+   * Thread to delete the tables
    *
    */
   private static final class DeleteFolderAndFiles implements Callable<Void> {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/core/src/main/java/org/carbondata/scan/filter/FilterUtil.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/filter/FilterUtil.java b/core/src/main/java/org/carbondata/scan/filter/FilterUtil.java
index a991d0f..d08b5a8 100644
--- a/core/src/main/java/org/carbondata/scan/filter/FilterUtil.java
+++ b/core/src/main/java/org/carbondata/scan/filter/FilterUtil.java
@@ -897,7 +897,7 @@ public final class FilterUtil {
     //    if (DataType.TIMESTAMP == carbonDimension.getDataType()) {
     //      return Integer.MAX_VALUE;
     //    }
-    // Get data from all the available slices of the cube
+    // Get data from all the available slices of the table
     if (null != dimCarinality) {
       return dimCarinality[carbonDimension.getKeyOrdinal()];
     }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/core/src/main/java/org/carbondata/scan/model/CarbonQueryPlan.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/carbondata/scan/model/CarbonQueryPlan.java b/core/src/main/java/org/carbondata/scan/model/CarbonQueryPlan.java
index 840aa02..d914b06 100644
--- a/core/src/main/java/org/carbondata/scan/model/CarbonQueryPlan.java
+++ b/core/src/main/java/org/carbondata/scan/model/CarbonQueryPlan.java
@@ -40,18 +40,14 @@ public class CarbonQueryPlan implements Serializable {
   private static final long serialVersionUID = -9036044826928017164L;
 
   /**
-   * Schema name , if user asks select * from datasight.employee.
-   * then datasight is the schame name.
-   * Remains null if the user does not select schema name.
+   * Database name
    */
-  private String schemaName;
+  private String databaseName;
 
   /**
-   * Cube name .
-   * if user asks select * from datasight.employee. then employee is the cube name.
-   * It is mandatory.
+   * Table name
    */
-  private String cubeName;
+  private String tableName;
 
   /**
    * List of dimensions.
@@ -110,23 +106,23 @@ public class CarbonQueryPlan implements Serializable {
   private boolean rawDetailQuery;
 
   /**
-   * Constructor created with cube name.
+   * Constructor created with table name.
    *
-   * @param cubeName
+   * @param tableName
    */
-  public CarbonQueryPlan(String cubeName) {
-    this.cubeName = cubeName;
+  public CarbonQueryPlan(String tableName) {
+    this.tableName = tableName;
   }
 
   /**
-   * Constructor created with schema name and cube name.
+   * Constructor created with database name and table name.
    *
-   * @param schemaName
-   * @param cubeName
+   * @param databaseName
+   * @param tableName
    */
-  public CarbonQueryPlan(String schemaName, String cubeName) {
-    this.cubeName = cubeName;
-    this.schemaName = schemaName;
+  public CarbonQueryPlan(String databaseName, String tableName) {
+    this.tableName = tableName;
+    this.databaseName = databaseName;
   }
 
   /**
@@ -160,17 +156,17 @@ public class CarbonQueryPlan implements Serializable {
   }
 
   /**
-   * @return the schemaName
+   * @return the databaseName
    */
-  public String getSchemaName() {
-    return schemaName;
+  public String getDatabaseName() {
+    return databaseName;
   }
 
   /**
-   * @return the cubeName
+   * @return the tableName
    */
-  public String getCubeName() {
-    return cubeName;
+  public String getTableName() {
+    return tableName;
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
----------------------------------------------------------------------
diff --git a/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala b/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
index 5269d05..728bdb0 100644
--- a/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
+++ b/examples/src/main/scala/org/carbondata/examples/GenerateDictionaryExample.scala
@@ -64,7 +64,7 @@ object GenerateDictionaryExample {
     val carbonRelation = CarbonEnv.getInstance(cc).carbonCatalog.
       lookupRelation1(Option(dataBaseName),
         tableName) (cc).asInstanceOf[CarbonRelation]
-    val carbonTable = carbonRelation.cubeMeta.carbonTable
+    val carbonTable = carbonRelation.tableMeta.carbonTable
     val dimensions = carbonTable.getDimensionByTableName(tableName.toLowerCase())
       .toArray.map(_.asInstanceOf[CarbonDimension])
     // scalastyle:off println

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java
----------------------------------------------------------------------
diff --git a/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java b/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java
index 3776918..0919e74 100644
--- a/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java
+++ b/hadoop/src/test/java/org/carbondata/hadoop/test/util/StoreCreator.java
@@ -131,8 +131,8 @@ public class StoreCreator {
       LoadModel loadModel = new LoadModel();
       String partitionId = "0";
       loadModel.setSchema(schema);
-      loadModel.setSchemaName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
-      loadModel.setCubeName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
+      loadModel.setDatabaseName(absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName());
+      loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
       loadModel.setTableName(absoluteTableIdentifier.getCarbonTableIdentifier().getTableName());
       loadModel.setFactFilePath(factFilePath);
       loadModel.setLoadMetadataDetails(new ArrayList<LoadMetadataDetails>());
@@ -334,9 +334,9 @@ public class StoreCreator {
     System.setProperty("KETTLE_HOME", kettleHomePath);
     new File(storeLocation).mkdirs();
     String outPutLoc = storeLocation + "/etl";
-    String schemaName = loadModel.getSchemaName();
-    String cubeName = loadModel.getCubeName();
-    String tempLocationKey = schemaName + '_' + cubeName + "_1";
+    String databaseName = loadModel.getDatabaseName();
+    String tableName = loadModel.getTableName();
+    String tempLocationKey = databaseName + '_' + tableName + "_1";
     CarbonProperties.getInstance().addProperty(tempLocationKey, storeLocation);
     CarbonProperties.getInstance().addProperty("store_output_location", outPutLoc);
     CarbonProperties.getInstance().addProperty("send.signal.load", "false");
@@ -349,18 +349,17 @@ public class StoreCreator {
     CarbonProperties.getInstance().addProperty("is.compressed.keyblock", "false");
     CarbonProperties.getInstance().addProperty("carbon.leaf.node.size", "120000");
 
-    String tableName = loadModel.getTableName();
     String fileNamePrefix = "";
 
     String graphPath =
-        outPutLoc + File.separator + loadModel.getSchemaName() + File.separator + tableName
+        outPutLoc + File.separator + loadModel.getDatabaseName() + File.separator + tableName
             + File.separator + 0 + File.separator + 1 + File.separator + tableName + ".ktr";
     File path = new File(graphPath);
     if (path.exists()) {
       path.delete();
     }
 
-    DataProcessTaskStatus schmaModel = new DataProcessTaskStatus(schemaName, cubeName, tableName);
+    DataProcessTaskStatus schmaModel = new DataProcessTaskStatus(databaseName, tableName);
     schmaModel.setCsvFilePath(loadModel.getFactFilePath());
     SchemaInfo info = new SchemaInfo();
     BlockDetails blockDetails = new BlockDetails(loadModel.getFactFilePath(),
@@ -368,8 +367,8 @@ public class StoreCreator {
     GraphGenerator.blockInfo.put("qwqwq", new BlockDetails[] { blockDetails });
     schmaModel.setBlocksID("qwqwq");
     schmaModel.setEscapeCharacter("\\");
-    info.setSchemaName(schemaName);
-    info.setCubeName(cubeName);
+    info.setDatabaseName(databaseName);
+    info.setTableName(tableName);
 
     generateGraph(schmaModel, info, loadModel.getTableName(), "0", loadModel.getSchema(), null,
         currentRestructNumber, loadModel.getLoadMetadataDetails());
@@ -380,11 +379,11 @@ public class StoreCreator {
             info, "0", loadModel.getSchema());
     //    LoadMetadataDetails[] loadDetails =
     //        CarbonUtil.readLoadMetadata(loadModel.schema.getCarbonTable().getMetaDataFilepath());
-    writeLoadMetadata(loadModel.schema, loadModel.getCubeName(), loadModel.getTableName(),
+    writeLoadMetadata(loadModel.schema, loadModel.getTableName(), loadModel.getTableName(),
         new ArrayList<LoadMetadataDetails>());
 
     String segLocation =
-        storeLocation + "/" + schemaName + "/" + tableName + "/Fact/Part0/Segment_0";
+        storeLocation + "/" + databaseName + "/" + tableName + "/Fact/Part0/Segment_0";
     File file = new File(segLocation);
     File factFile = null;
     File[] folderList = file.listFiles();
@@ -408,8 +407,8 @@ public class StoreCreator {
     }
   }
 
-  public static void writeLoadMetadata(CarbonDataLoadSchema schema, String schemaName,
-      String cubeName, List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
+  public static void writeLoadMetadata(CarbonDataLoadSchema schema, String databaseName,
+      String tableName, List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
     LoadMetadataDetails loadMetadataDetails = new LoadMetadataDetails();
     loadMetadataDetails.setTimestamp(readCurrentTime());
     loadMetadataDetails.setLoadStatus("SUCCESS");
@@ -511,8 +510,7 @@ public class StoreCreator {
 
     private CarbonDataLoadSchema schema;
     private String tableName;
-    private String cubeName;
-    private String schemaName;
+    private String databaseName;
     private List<LoadMetadataDetails> loadMetaDetail;
     private String factFilePath;
 
@@ -536,12 +534,8 @@ public class StoreCreator {
       return tableName;
     }
 
-    public String getCubeName() {
-      return cubeName;
-    }
-
-    public String getSchemaName() {
-      return schemaName;
+    public String getDatabaseName() {
+      return databaseName;
     }
 
     public void setLoadMetadataDetails(List<LoadMetadataDetails> loadMetaDetail) {
@@ -556,12 +550,8 @@ public class StoreCreator {
       this.tableName = tableName;
     }
 
-    public void setCubeName(String cubeName) {
-      this.cubeName = cubeName;
-    }
-
-    public void setSchemaName(String schemaName) {
-      this.schemaName = schemaName;
+    public void setDatabaseName(String databaseName) {
+      this.databaseName = databaseName;
     }
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
index e2cd277..d001a59 100644
--- a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
+++ b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CarbonCompactionExecutor.java
@@ -55,7 +55,7 @@ public class CarbonCompactionExecutor {
   private final Map<String, List<DataFileFooter>> dataFileMetadataSegMapping;
   private QueryExecutor queryExecutor;
   private final SegmentProperties destinationSegProperties;
-  private final String schemaName;
+  private final String databaseName;
   private final String factTableName;
   private final Map<String, TaskBlockInfo> segmentMapping;
   private final String storePath;
@@ -70,13 +70,13 @@ public class CarbonCompactionExecutor {
    * Constructor
    * @param segmentMapping
    * @param segmentProperties
-   * @param schemaName
+   * @param databaseName
    * @param factTableName
    * @param storePath
    * @param carbonTable
    */
   public CarbonCompactionExecutor(Map<String, TaskBlockInfo> segmentMapping,
-      SegmentProperties segmentProperties, String schemaName, String factTableName,
+      SegmentProperties segmentProperties, String databaseName, String factTableName,
       String storePath, CarbonTable carbonTable,
       Map<String, List<DataFileFooter>> dataFileMetadataSegMapping) {
 
@@ -84,7 +84,7 @@ public class CarbonCompactionExecutor {
 
     this.destinationSegProperties = segmentProperties;
 
-    this.schemaName = schemaName;
+    this.databaseName = databaseName;
 
     this.factTableName = factTableName;
 

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CompactionCallable.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CompactionCallable.java b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CompactionCallable.java
index 7289e61..16b4924 100644
--- a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CompactionCallable.java
+++ b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/CompactionCallable.java
@@ -38,14 +38,14 @@ public class CompactionCallable implements Callable<Void> {
   private final String storeLocation;
   private final CarbonTable carbonTable;
   private final String kettleHomePath;
-  private final Long cubeCreationTime;
+  private final Long tableCreationTime;
   private final List<LoadMetadataDetails> loadsToMerge;
   private final SQLContext sqlContext;
   private final CarbonLoadModel carbonLoadModel;
 
   public CompactionCallable(String hdfsStoreLocation, CarbonLoadModel carbonLoadModel,
       Partitioner partitioner, String storeLocation, CarbonTable carbonTable, String kettleHomePath,
-      Long cubeCreationTime, List<LoadMetadataDetails> loadsToMerge, SQLContext sqlContext) {
+      Long tableCreationTime, List<LoadMetadataDetails> loadsToMerge, SQLContext sqlContext) {
 
     this.hdfsStoreLocation = hdfsStoreLocation;
     this.carbonLoadModel = carbonLoadModel;
@@ -53,7 +53,7 @@ public class CompactionCallable implements Callable<Void> {
     this.storeLocation = storeLocation;
     this.carbonTable = carbonTable;
     this.kettleHomePath = kettleHomePath;
-    this.cubeCreationTime = cubeCreationTime;
+    this.tableCreationTime = tableCreationTime;
     this.loadsToMerge = loadsToMerge;
     this.sqlContext = sqlContext;
   }
@@ -61,7 +61,7 @@ public class CompactionCallable implements Callable<Void> {
   @Override public Void call() throws Exception {
 
     Compactor.triggerCompaction(hdfsStoreLocation, carbonLoadModel, partitioner, storeLocation,
-        carbonTable, kettleHomePath, cubeCreationTime, loadsToMerge, sqlContext);
+        carbonTable, kettleHomePath, tableCreationTime, loadsToMerge, sqlContext);
     return null;
 
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
index 54e32a0..4098dca 100644
--- a/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
+++ b/integration/spark/src/main/java/org/carbondata/integration/spark/merger/RowResultMerger.java
@@ -59,7 +59,7 @@ import org.carbondata.spark.load.CarbonLoadModel;
  */
 public class RowResultMerger {
 
-  private final String schemaName;
+  private final String databaseName;
   private final String tableName;
   private final String tempStoreLocation;
   private final int measureCount;
@@ -78,9 +78,9 @@ public class RowResultMerger {
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(RowResultMerger.class.getName());
 
-  public RowResultMerger(List<RawResultIterator> iteratorList, String schemaName, String tableName,
-      SegmentProperties segProp, String tempStoreLocation, CarbonLoadModel loadModel,
-      int[] colCardinality) {
+  public RowResultMerger(List<RawResultIterator> iteratorList, String databaseName,
+      String tableName, SegmentProperties segProp, String tempStoreLocation,
+      CarbonLoadModel loadModel, int[] colCardinality) {
 
     this.rawResultIteratorList = iteratorList;
     // create the List of RawResultIterator.
@@ -97,7 +97,7 @@ public class RowResultMerger {
       LOGGER.error("Error while new File(tempStoreLocation).mkdirs() ");
     }
 
-    this.schemaName = schemaName;
+    this.databaseName = databaseName;
     this.tableName = tableName;
 
     this.measureCount = segprop.getMeasures().size();
@@ -212,7 +212,7 @@ public class RowResultMerger {
    */
   private CarbonFactDataHandlerModel getCarbonFactDataHandlerModel(CarbonLoadModel loadModel) {
     CarbonFactDataHandlerModel carbonFactDataHandlerModel = new CarbonFactDataHandlerModel();
-    carbonFactDataHandlerModel.setDatabaseName(schemaName);
+    carbonFactDataHandlerModel.setDatabaseName(databaseName);
     carbonFactDataHandlerModel.setTableName(tableName);
     carbonFactDataHandlerModel.setMeasureCount(segprop.getMeasures().size());
     carbonFactDataHandlerModel.setCompactionFlow(true);
@@ -225,7 +225,7 @@ public class RowResultMerger {
     carbonFactDataHandlerModel.setDimensionCount(
         segprop.getDimensions().size() - carbonFactDataHandlerModel.getNoDictionaryCount());
     CarbonTable carbonTable = CarbonMetadata.getInstance()
-        .getCarbonTable(schemaName + CarbonCommonConstants.UNDERSCORE + tableName);
+        .getCarbonTable(databaseName + CarbonCommonConstants.UNDERSCORE + tableName);
     List<ColumnSchema> wrapperColumnSchema = CarbonUtil
         .getColumnSchemaList(carbonTable.getDimensionByTableName(tableName),
             carbonTable.getMeasureByTableName(tableName));
@@ -246,7 +246,7 @@ public class RowResultMerger {
     carbonFactDataHandlerModel.setFactDimLens(segprop.getDimColumnsCardinality());
 
     String carbonDataDirectoryPath =
-        checkAndCreateCarbonStoreLocation(this.factStoreLocation, schemaName, tableName,
+        checkAndCreateCarbonStoreLocation(this.factStoreLocation, databaseName, tableName,
             loadModel.getPartitionId(), loadModel.getSegmentId());
     carbonFactDataHandlerModel.setCarbonDataDirectoryPath(carbonDataDirectoryPath);
 
@@ -258,11 +258,11 @@ public class RowResultMerger {
    *
    * @return data directory path
    */
-  private String checkAndCreateCarbonStoreLocation(String factStoreLocation, String schemaName,
+  private String checkAndCreateCarbonStoreLocation(String factStoreLocation, String databaseName,
       String tableName, String partitionId, String segmentId) {
     String carbonStorePath = factStoreLocation;
     CarbonTable carbonTable = CarbonMetadata.getInstance()
-        .getCarbonTable(schemaName + CarbonCommonConstants.UNDERSCORE + tableName);
+        .getCarbonTable(databaseName + CarbonCommonConstants.UNDERSCORE + tableName);
     CarbonTableIdentifier carbonTableIdentifier = carbonTable.getCarbonTableIdentifier();
     CarbonTablePath carbonTablePath =
         CarbonStorePath.getCarbonTablePath(carbonStorePath, carbonTableIdentifier);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java b/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
index 52f8fb7..ae1cf73 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/load/CarbonLoaderUtil.java
@@ -162,8 +162,7 @@ public final class CarbonLoaderUtil {
       path.delete();
     }
 
-    DataProcessTaskStatus schmaModel =
-        new DataProcessTaskStatus(databaseName, tableName, loadModel.getTableName());
+    DataProcessTaskStatus schmaModel = new DataProcessTaskStatus(databaseName, tableName);
     schmaModel.setCsvFilePath(loadModel.getFactFilePath());
     schmaModel.setDimCSVDirLoc(loadModel.getDimFolderPath());
     if (loadModel.isDirectLoad()) {
@@ -177,8 +176,8 @@ public final class CarbonLoaderUtil {
     schmaModel.setEscapeCharacter(loadModel.getEscapeChar());
     SchemaInfo info = new SchemaInfo();
 
-    info.setSchemaName(databaseName);
-    info.setCubeName(tableName);
+    info.setDatabaseName(databaseName);
+    info.setTableName(tableName);
     info.setAutoAggregateRequest(loadModel.isAggLoadRequest());
     info.setComplexDelimiterLevel1(loadModel.getComplexDelimiterLevel1());
     info.setComplexDelimiterLevel2(loadModel.getComplexDelimiterLevel2());
@@ -191,12 +190,12 @@ public final class CarbonLoaderUtil {
             info, loadModel.getPartitionId(), loadModel.getCarbonDataLoadSchema());
   }
 
-  public static String[] getStorelocs(String schemaName, String cubeName, String factTableName,
+  public static String[] getStorelocs(String databaseName, String tableName, String factTableName,
       String hdfsStoreLocation, int currentRestructNumber) {
     String[] loadFolders;
 
     String baseStorelocation =
-        hdfsStoreLocation + File.separator + schemaName + File.separator + cubeName;
+        hdfsStoreLocation + File.separator + databaseName + File.separator + tableName;
 
     String factStorepath =
         baseStorelocation + File.separator + CarbonCommonConstants.RESTRUCTRE_FOLDER
@@ -231,11 +230,11 @@ public final class CarbonLoaderUtil {
     return activeSlices;
   }
 
-  public static String getAggLoadFolderLocation(String loadFolderName, String schemaName,
-      String cubeName, String aggTableName, String hdfsStoreLocation, int currentRestructNumber) {
+  public static String getAggLoadFolderLocation(String loadFolderName, String databaseName,
+      String tableName, String aggTableName, String hdfsStoreLocation, int currentRestructNumber) {
     for (int i = currentRestructNumber; i >= 0; i--) {
       String aggTableLocation =
-          getTableLocation(schemaName, cubeName, aggTableName, hdfsStoreLocation, i);
+          getTableLocation(databaseName, tableName, aggTableName, hdfsStoreLocation, i);
       String aggStorepath = aggTableLocation + File.separator + loadFolderName;
       try {
         if (FileFactory.isFileExist(aggStorepath, FileFactory.getFileType(aggStorepath))) {
@@ -248,26 +247,26 @@ public final class CarbonLoaderUtil {
     return null;
   }
 
-  public static String getTableLocation(String schemaName, String cubeName, String aggTableName,
+  public static String getTableLocation(String databaseName, String tableName, String aggTableName,
       String hdfsStoreLocation, int currentRestructNumber) {
     String baseStorelocation =
-        hdfsStoreLocation + File.separator + schemaName + File.separator + cubeName;
+        hdfsStoreLocation + File.separator + databaseName + File.separator + tableName;
     String aggTableLocation =
         baseStorelocation + File.separator + CarbonCommonConstants.RESTRUCTRE_FOLDER
             + currentRestructNumber + File.separator + aggTableName;
     return aggTableLocation;
   }
 
-  public static void deleteTable(int partitionCount, String schemaName, String cubeName,
+  public static void deleteTable(int partitionCount, String databaseName, String tableName,
       String aggTableName, String hdfsStoreLocation, int currentRestructNumber) {
     String aggTableLoc = null;
-    String partitionSchemaName = null;
-    String partitionCubeName = null;
+    String partitionDatabaseName = null;
+    String partitionTableName = null;
     for (int i = 0; i < partitionCount; i++) {
-      partitionSchemaName = schemaName + '_' + i;
-      partitionCubeName = cubeName + '_' + i;
+      partitionDatabaseName = databaseName + '_' + i;
+      partitionTableName = tableName + '_' + i;
       for (int j = currentRestructNumber; j >= 0; j--) {
-        aggTableLoc = getTableLocation(partitionSchemaName, partitionCubeName, aggTableName,
+        aggTableLoc = getTableLocation(partitionDatabaseName, partitionTableName, aggTableName,
             hdfsStoreLocation, j);
         deleteStorePath(aggTableLoc);
       }
@@ -285,16 +284,16 @@ public final class CarbonLoaderUtil {
     }
   }
 
-  public static void deleteSlice(int partitionCount, String schemaName, String cubeName,
-      String tableName, String hdfsStoreLocation, int currentRestructNumber, String loadFolder) {
+  public static void deleteSlice(int partitionCount, String databaseName, String tableName,
+      String hdfsStoreLocation, int currentRestructNumber, String loadFolder) {
     String tableLoc = null;
-    String partitionSchemaName = null;
-    String partitionCubeName = null;
+    String partitionDatabaseName = null;
+    String partitionTableName = null;
     for (int i = 0; i < partitionCount; i++) {
-      partitionSchemaName = schemaName + '_' + i;
-      partitionCubeName = cubeName + '_' + i;
+      partitionDatabaseName = databaseName + '_' + i;
+      partitionTableName = tableName + '_' + i;
       tableLoc =
-          getTableLocation(partitionSchemaName, partitionCubeName, tableName, hdfsStoreLocation,
+          getTableLocation(partitionDatabaseName, partitionTableName, tableName, hdfsStoreLocation,
               currentRestructNumber);
       tableLoc = tableLoc + File.separator + loadFolder;
       deleteStorePath(tableLoc);
@@ -416,7 +415,7 @@ public final class CarbonLoaderUtil {
     return updatedSlices;
   }
 
-  public static void removeSliceFromMemory(String schemaName, String cubeName, String loadName) {
+  public static void removeSliceFromMemory(String databaseName, String tableName, String loadName) {
     // TODO: Remove from memory
   }
 
@@ -678,8 +677,8 @@ public final class CarbonLoaderUtil {
    * API will provide the load folder path for the store inorder to store the same
    * in the metadata.
    */
-  public static String getLoadFolderPath(CarbonLoadModel loadModel, String cubeName,
-      String schemaName, int currentRestructNumber) {
+  public static String getLoadFolderPath(CarbonLoadModel loadModel, String tableName,
+      String databaseName, int currentRestructNumber) {
 
     //CHECKSTYLE:OFF    Approval No:Approval-V1R2C10_005
 
@@ -687,9 +686,9 @@ public final class CarbonLoaderUtil {
         Boolean.valueOf(CarbonProperties.getInstance().getProperty("dataload.hdfs.copy", "true"));
 
     // CHECKSTYLE:ON
-    if (null == cubeName && null == schemaName) {
-      schemaName = loadModel.getDatabaseName();
-      cubeName = loadModel.getTableName();
+    if (null == tableName && null == databaseName) {
+      databaseName = loadModel.getDatabaseName();
+      tableName = loadModel.getTableName();
     }
     String factTable = loadModel.getTableName();
     String hdfsLoadedTable = null;
@@ -703,7 +702,7 @@ public final class CarbonLoaderUtil {
       if (!hdfsLocation.equals(localStore)) {
         String hdfsStoreLocation = hdfsLocation;
         hdfsStoreLocation =
-            hdfsStoreLocation + File.separator + schemaName + File.separator + cubeName;
+            hdfsStoreLocation + File.separator + databaseName + File.separator + tableName;
 
         int rsCounter = currentRestructNumber;
         if (rsCounter == -1) {
@@ -804,7 +803,7 @@ public final class CarbonLoaderUtil {
   }
 
   public static void writeLoadMetadata(CarbonDataLoadSchema schema,
-      String schemaName, String tableName,
+      String databaseName, String tableName,
       List<LoadMetadataDetails> listOfLoadFolderDetails) throws IOException {
     CarbonTablePath carbonTablePath = CarbonStorePath
         .getCarbonTablePath(schema.getCarbonTable().getStorePath(),
@@ -882,8 +881,8 @@ public final class CarbonLoaderUtil {
     boolean copyStore =
         Boolean.valueOf(CarbonProperties.getInstance().getProperty("dataload.hdfs.copy", "true"));
 
-    String schemaName = loadModel.getDatabaseName();
-    String cubeName = loadModel.getTableName();
+    String databaseName = loadModel.getDatabaseName();
+    String tableName = loadModel.getTableName();
     String factTable = loadModel.getTableName();
     String aggTableName = loadModel.getAggTableName();
 
@@ -895,23 +894,23 @@ public final class CarbonLoaderUtil {
           .getProperty(CarbonCommonConstants.STORE_LOCATION,
               CarbonCommonConstants.STORE_LOCATION_DEFAULT_VAL);
       if (!loadModel.isAggLoadRequest()) {
-        copyMergeToHDFS(schemaName, cubeName, factTable, hdfsLocation, localStore,
+        copyMergeToHDFS(databaseName, tableName, factTable, hdfsLocation, localStore,
             currentRestructNumber, mergedLoadName);
       }
       if (null != aggTableName) {
-        copyMergeToHDFS(schemaName, cubeName, aggTableName, hdfsLocation, localStore,
+        copyMergeToHDFS(databaseName, tableName, aggTableName, hdfsLocation, localStore,
             currentRestructNumber, mergedLoadName);
       }
       try {
         CarbonUtil.deleteFoldersAndFiles(new File[] {
-            new File(localStore + File.separator + schemaName + File.separator + cubeName) });
+            new File(localStore + File.separator + databaseName + File.separator + tableName) });
       } catch (CarbonUtilException e) {
         LOGGER.error("Error while CarbonUtil.deleteFoldersAndFiles ");
       }
     }
   }
 
-  public static void copyMergeToHDFS(String schemaName, String cubeName, String factTable,
+  public static void copyMergeToHDFS(String databaseName, String tableName, String factTable,
       String hdfsLocation, String localStore, int currentRestructNumber, String mergedLoadName) {
     try {
       //If the hdfs store and the local store configured differently, then copy
@@ -921,7 +920,7 @@ public final class CarbonLoaderUtil {
          */
         String currentloadedStore = localStore;
         currentloadedStore =
-            currentloadedStore + File.separator + schemaName + File.separator + cubeName;
+            currentloadedStore + File.separator + databaseName + File.separator + tableName;
 
         int rsCounter = currentRestructNumber;
 
@@ -952,7 +951,7 @@ public final class CarbonLoaderUtil {
         //Identify the Load_X folder in the HDFS store
         String hdfsStoreLocation = hdfsLocation;
         hdfsStoreLocation =
-            hdfsStoreLocation + File.separator + schemaName + File.separator + cubeName;
+            hdfsStoreLocation + File.separator + databaseName + File.separator + tableName;
 
         rsCounter = currentRestructNumber;
         if (rsCounter == -1) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/CSVFilePartitioner.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/CSVFilePartitioner.java b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/CSVFilePartitioner.java
index 2a466b7..ce9a654 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/CSVFilePartitioner.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/CSVFilePartitioner.java
@@ -85,17 +85,10 @@ public class CSVFilePartitioner {
     return partialSuccess;
   }
 
-  /**
-   * @param sourceFilePath - Source raw data file in local disk
-   * @param targetFolder   - Target folder to save the partitioned files
-   * @param nodes
-   * @param properties
-   * @param i
-   */
-  @Deprecated public void splitFile(String schemaName, String cubeName, List<String> sourceFilePath,
-      String targetFolder, List<String> nodes, int partitionCount, String[] partitionColumn,
-      String[] requiredColumns, String delimiter, String quoteChar, String fileHeader,
-      String escapeChar, boolean multiLine) throws Exception {
+  @Deprecated public void splitFile(String databaseName, String tableName,
+      List<String> sourceFilePath, String targetFolder, List<String> nodes, int partitionCount,
+      String[] partitionColumn, String[] requiredColumns, String delimiter, String quoteChar,
+      String fileHeader, String escapeChar, boolean multiLine) throws Exception {
     LOGGER
         .info("Processing file split: " + sourceFilePath);
 
@@ -107,9 +100,9 @@ public class CSVFilePartitioner {
     HashMap<Partition, CSVWriter> outputStreamsMap =
         new HashMap<Partition, CSVWriter>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
 
-    String key = schemaName + '_' + cubeName;
+    String key = databaseName + '_' + tableName;
     badRecordslogger = new BadRecordslogger(key, "Partition_" + System.currentTimeMillis() + ".log",
-        getBadLogStoreLocation("partition/" + schemaName + '/' + cubeName));
+        getBadLogStoreLocation("partition/" + databaseName + '/' + tableName));
 
     CSVReader dataInputStream = null;
 
@@ -290,8 +283,8 @@ public class CSVFilePartitioner {
         partialSuccess = true;
         skippedLines++;
         badRecordslogger.addBadRecordsToBilder(record, record.length,
-            "No. of columns not matched with cube columns", null);
-        LOGGER.error("BAD Record Found: No. of columns not matched with cube columns, "
+            "No. of columns not matched with table columns", null);
+        LOGGER.error("BAD Record Found: No. of columns not matched with table columns, "
             + "Skipping line: (" + (recordCounter + 1) + ") in File :" + fileAbsolutePath);
       } catch (Exception e) {
         partialSuccess = true;

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/QueryPartitionHelper.java b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
index c14515b..50b9b53 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/partition/api/impl/QueryPartitionHelper.java
@@ -97,7 +97,7 @@ public final class QueryPartitionHelper {
 
   }
 
-  private void checkInitialization(String cubeUniqueName, Partitioner partitioner) {
+  private void checkInitialization(String tableUniqueName, Partitioner partitioner) {
     //Initialise if not done earlier
 
     //String nodeListString = null;
@@ -114,7 +114,7 @@ public final class QueryPartitionHelper {
       LOGGER.info("nodeList : " + Arrays.toString(partitioner.nodeList()));
     }
 
-    if (partitionerMap.get(cubeUniqueName) == null) {
+    if (partitionerMap.get(tableUniqueName) == null) {
       DataPartitioner dataPartitioner;
       try {
         dataPartitioner =
@@ -124,8 +124,8 @@ public final class QueryPartitionHelper {
         List<Partition> partitions = dataPartitioner.getAllPartitions();
         DefaultLoadBalancer loadBalancer =
             new DefaultLoadBalancer(Arrays.asList(partitioner.nodeList()), partitions);
-        partitionerMap.put(cubeUniqueName, dataPartitioner);
-        loadBalancerMap.put(cubeUniqueName, loadBalancer);
+        partitionerMap.put(tableUniqueName, dataPartitioner);
+        loadBalancerMap.put(tableUniqueName, loadBalancer);
       } catch (ClassNotFoundException e) {
         LOGGER.error(e,
             e.getMessage());
@@ -143,47 +143,47 @@ public final class QueryPartitionHelper {
    * Get partitions applicable for query based on filters applied in query
    */
   public List<Partition> getPartitionsForQuery(CarbonQueryPlan queryPlan, Partitioner partitioner) {
-    String cubeUniqueName = queryPlan.getSchemaName() + '_' + queryPlan.getCubeName();
-    checkInitialization(cubeUniqueName, partitioner);
+    String tableUniqueName = queryPlan.getDatabaseName() + '_' + queryPlan.getTableName();
+    checkInitialization(tableUniqueName, partitioner);
 
-    DataPartitioner dataPartitioner = partitionerMap.get(cubeUniqueName);
+    DataPartitioner dataPartitioner = partitionerMap.get(tableUniqueName);
 
     List<Partition> queryPartitions = dataPartitioner.getPartitions(queryPlan);
     return queryPartitions;
   }
 
-  public List<Partition> getAllPartitions(String schemaName, String cubeName,
+  public List<Partition> getAllPartitions(String databaseName, String tableName,
       Partitioner partitioner) {
-    String cubeUniqueName = schemaName + '_' + cubeName;
-    checkInitialization(cubeUniqueName, partitioner);
+    String tableUniqueName = databaseName + '_' + tableName;
+    checkInitialization(tableUniqueName, partitioner);
 
-    DataPartitioner dataPartitioner = partitionerMap.get(cubeUniqueName);
+    DataPartitioner dataPartitioner = partitionerMap.get(tableUniqueName);
 
     return dataPartitioner.getAllPartitions();
   }
 
-  public void removePartition(String schemaName, String cubeName) {
-    String cubeUniqueName = schemaName + '_' + cubeName;
-    partitionerMap.remove(cubeUniqueName);
+  public void removePartition(String databaseName, String tableName) {
+    String tableUniqueName = databaseName + '_' + tableName;
+    partitionerMap.remove(tableUniqueName);
   }
 
   /**
    * Get the node name where the partition is assigned to.
    */
-  public String getLocation(Partition partition, String schemaName, String cubeName,
+  public String getLocation(Partition partition, String databaseName, String tableName,
       Partitioner partitioner) {
-    String cubeUniqueName = schemaName + '_' + cubeName;
-    checkInitialization(cubeUniqueName, partitioner);
+    String tableUniqueName = databaseName + '_' + tableName;
+    checkInitialization(tableUniqueName, partitioner);
 
-    DefaultLoadBalancer loadBalancer = loadBalancerMap.get(cubeUniqueName);
+    DefaultLoadBalancer loadBalancer = loadBalancerMap.get(tableUniqueName);
     return loadBalancer.getNodeForPartitions(partition);
   }
 
-  public String[] getPartitionedColumns(String schemaName, String cubeName,
+  public String[] getPartitionedColumns(String databaseName, String tableName,
       Partitioner partitioner) {
-    String cubeUniqueName = schemaName + '_' + cubeName;
-    checkInitialization(cubeUniqueName, partitioner);
-    DataPartitioner dataPartitioner = partitionerMap.get(cubeUniqueName);
+    String tableUniqueName = databaseName + '_' + tableName;
+    checkInitialization(tableUniqueName, partitioner);
+    DataPartitioner dataPartitioner = partitionerMap.get(tableUniqueName);
     return dataPartitioner.getPartitionedColumns();
   }
 }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/integration/spark/src/main/java/org/carbondata/spark/util/CarbonQueryUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/util/CarbonQueryUtil.java b/integration/spark/src/main/java/org/carbondata/spark/util/CarbonQueryUtil.java
index b9d5fce..c2cbbb5 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/util/CarbonQueryUtil.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/util/CarbonQueryUtil.java
@@ -56,14 +56,14 @@ public final class CarbonQueryUtil {
   /**
    * It creates the one split for each region server.
    */
-  public static synchronized TableSplit[] getTableSplits(String schemaName, String cubeName,
+  public static synchronized TableSplit[] getTableSplits(String databaseName, String tableName,
       CarbonQueryPlan queryPlan, Partitioner partitioner) throws IOException {
 
     //Just create splits depends on locations of region servers
     List<Partition> allPartitions = null;
     if (queryPlan == null) {
       allPartitions =
-          QueryPartitionHelper.getInstance().getAllPartitions(schemaName, cubeName, partitioner);
+          QueryPartitionHelper.getInstance().getAllPartitions(databaseName, tableName, partitioner);
     } else {
       allPartitions =
           QueryPartitionHelper.getInstance().getPartitionsForQuery(queryPlan, partitioner);
@@ -74,7 +74,7 @@ public final class CarbonQueryUtil {
       List<String> locations = new ArrayList<String>(CarbonCommonConstants.CONSTANT_SIZE_TEN);
       Partition partition = allPartitions.get(i);
       String location = QueryPartitionHelper.getInstance()
-          .getLocation(partition, schemaName, cubeName, partitioner);
+          .getLocation(partition, databaseName, tableName, partitioner);
       locations.add(location);
       splits[i].setPartition(partition);
       splits[i].setLocations(locations);

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/integration/spark/src/main/java/org/carbondata/spark/util/LoadMetadataUtil.java
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/java/org/carbondata/spark/util/LoadMetadataUtil.java b/integration/spark/src/main/java/org/carbondata/spark/util/LoadMetadataUtil.java
index d35eb87..d46cbdf 100644
--- a/integration/spark/src/main/java/org/carbondata/spark/util/LoadMetadataUtil.java
+++ b/integration/spark/src/main/java/org/carbondata/spark/util/LoadMetadataUtil.java
@@ -45,12 +45,12 @@ public final class LoadMetadataUtil {
   }
 
   public static boolean isLoadDeletionRequired(CarbonLoadModel loadModel) {
-    CarbonTable cube = org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance()
+    CarbonTable table = org.carbondata.core.carbon.metadata.CarbonMetadata.getInstance()
         .getCarbonTable(loadModel.getDatabaseName() + '_' + loadModel.getTableName());
 
-    String metaDataLocation = cube.getMetaDataFilepath();
+    String metaDataLocation = table.getMetaDataFilepath();
     SegmentStatusManager segmentStatusManager =
-        new SegmentStatusManager(cube.getAbsoluteTableIdentifier());
+        new SegmentStatusManager(table.getAbsoluteTableIdentifier());
     LoadMetadataDetails[] details = segmentStatusManager.readLoadMetadata(metaDataLocation);
     if (details != null && details.length != 0) {
       for (LoadMetadataDetails oneRow : details) {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
index c7333cf..2f5ec6f 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCatalystOperators.scala
@@ -48,9 +48,9 @@ object getDB {
 }
 
 /**
- * Shows Loads in a cube
+ * Shows Loads in a table
  */
-case class ShowLoadsCommand(schemaNameOp: Option[String], cube: String, limit: Option[String])
+case class ShowLoadsCommand(databaseNameOp: Option[String], table: String, limit: Option[String])
   extends LogicalPlan with Command {
 
   override def children: Seq[LogicalPlan] = Seq.empty

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
index d47d255..1d46081 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonContext.scala
@@ -128,8 +128,8 @@ object CarbonContext {
   val datasourceShortName: String = "carbondata"
 
   /**
-   * @param schemaName - Schema Name
-   * @param cubeName   - Cube Name
+   * @param databaseName - Database Name
+   * @param tableName   - Table Name
    * @param factPath   - Raw CSV data path
    * @param targetPath - Target path where the file will be split as per partition
    * @param delimiter  - default file delimiter is comma(,)
@@ -147,8 +147,8 @@ object CarbonContext {
    *                   lines for end of quote & skip all lines if end of quote not found.
    */
   final def partitionData(
-      schemaName: String = null,
-      cubeName: String,
+      databaseName: String = null,
+      tableName: String,
       factPath: String,
       targetPath: String,
       delimiter: String = ",",
@@ -157,11 +157,11 @@ object CarbonContext {
       escapeChar: String = null,
       multiLine: Boolean = false)(hiveContext: HiveContext): String = {
     updateCarbonPorpertiesPath(hiveContext)
-    var schemaNameLocal = schemaName
-    if (schemaNameLocal == null) {
-      schemaNameLocal = "default"
+    var databaseNameLocal = databaseName
+    if (databaseNameLocal == null) {
+      databaseNameLocal = "default"
     }
-    val partitionDataClass = PartitionData(schemaName, cubeName, factPath, targetPath, delimiter,
+    val partitionDataClass = PartitionData(databaseName, tableName, factPath, targetPath, delimiter,
       quoteChar, fileHeader, escapeChar, multiLine)
     partitionDataClass.run(hiveContext)
     partitionDataClass.partitionStatus

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
index 5613d71..35d989a 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDatasourceRelation.scala
@@ -147,10 +147,10 @@ private[sql] case class CarbonDatasourceRelation(
   def sqlContext: SQLContext = context
 
   override val sizeInBytes: Long = {
-    val tablePath = carbonRelation.cubeMeta.storePath + CarbonCommonConstants.FILE_SEPARATOR +
-                    carbonRelation.cubeMeta.carbonTableIdentifier.getDatabaseName +
+    val tablePath = carbonRelation.tableMeta.storePath + CarbonCommonConstants.FILE_SEPARATOR +
+                    carbonRelation.tableMeta.carbonTableIdentifier.getDatabaseName +
                     CarbonCommonConstants.FILE_SEPARATOR +
-                    carbonRelation.cubeMeta.carbonTableIdentifier.getTableName
+                    carbonRelation.tableMeta.carbonTableIdentifier.getTableName
     val fileType = FileFactory.getFileType
     if(FileFactory.isFileExist(tablePath, fileType)) {
       FileFactory.getDirectorySize(tablePath)
@@ -161,18 +161,16 @@ private[sql] case class CarbonDatasourceRelation(
 }
 
 /**
- * Represents logical plan for one carbon cube
+ * Represents logical plan for one carbon table
  */
 case class CarbonRelation(
-    schemaName: String,
-    cubeName: String,
+    databaseName: String,
+    tableName: String,
     metaData: CarbonMetaData,
-    cubeMeta: TableMeta,
+    tableMeta: TableMeta,
     alias: Option[String])(@transient sqlContext: SQLContext)
   extends LeafNode with MultiInstanceRelation {
 
-  def tableName: String = cubeName
-
   def recursiveMethod(dimName: String, childDim: CarbonDimension): String = {
     childDim.getDataType.toString.toLowerCase match {
       case "array" => s"${
@@ -212,13 +210,13 @@ case class CarbonRelation(
   }
 
   override def newInstance(): LogicalPlan = {
-    CarbonRelation(schemaName, cubeName, metaData, cubeMeta, alias)(sqlContext)
+    CarbonRelation(databaseName, tableName, metaData, tableMeta, alias)(sqlContext)
       .asInstanceOf[this.type]
   }
 
   val dimensionsAttr = {
     val sett = new LinkedHashSet(
-      cubeMeta.carbonTable.getDimensionByTableName(cubeMeta.carbonTableIdentifier.getTableName)
+      tableMeta.carbonTable.getDimensionByTableName(tableMeta.carbonTableIdentifier.getTableName)
         .asScala.asJava)
     sett.asScala.toSeq.filter(!_.getColumnSchema.isInvisible).map(dim => {
       val output: DataType = metaData.carbonTable
@@ -239,10 +237,10 @@ case class CarbonRelation(
   }
 
   val measureAttr = {
-    val factTable = cubeMeta.carbonTable.getFactTableName
+    val factTable = tableMeta.carbonTable.getFactTableName
     new LinkedHashSet(
-      cubeMeta.carbonTable.
-        getMeasureByTableName(cubeMeta.carbonTable.getFactTableName).
+      tableMeta.carbonTable.
+        getMeasureByTableName(tableMeta.carbonTable.getFactTableName).
         asScala.asJava).asScala.toSeq.filter(!_.getColumnSchema.isInvisible)
         .map(x => AttributeReference(x.getColName, CarbonMetastoreTypes.toDataType(
         metaData.carbonTable.getMeasureByName(factTable, x.getColName).getDataType.toString
@@ -263,7 +261,7 @@ case class CarbonRelation(
   override def equals(other: Any): Boolean = {
     other match {
       case p: CarbonRelation =>
-        p.schemaName == schemaName && p.output == output && p.cubeName == cubeName
+        p.databaseName == databaseName && p.output == output && p.tableName == tableName
       case _ => false
     }
   }

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
index 782b87c..21aded4 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonOperators.scala
@@ -50,7 +50,7 @@ case class CarbonScan(
   val unprocessedExprs = new ArrayBuffer[Expression]()
 
   val buildCarbonPlan: CarbonQueryPlan = {
-    val plan: CarbonQueryPlan = new CarbonQueryPlan(relationRaw.schemaName, relationRaw.tableName)
+    val plan: CarbonQueryPlan = new CarbonQueryPlan(relationRaw.databaseName, relationRaw.tableName)
 
     val dimensions = carbonTable.getDimensionByTableName(carbonTable.getFactTableName)
     val measures = carbonTable.getMeasureByTableName(carbonTable.getFactTableName)
@@ -168,17 +168,17 @@ case class CarbonScan(
     // setting queryid
     buildCarbonPlan.setQueryId(ocRaw.getConf("queryId", System.nanoTime() + ""))
 
-    val cubeCreationTime = carbonCatalog
-      .getCubeCreationTime(relationRaw.schemaName, relationRaw.tableName)
+    val tableCreationTime = carbonCatalog
+      .getTableCreationTime(relationRaw.databaseName, relationRaw.tableName)
     val schemaLastUpdatedTime = carbonCatalog
-      .getSchemaLastUpdatedTime(relationRaw.schemaName, relationRaw.tableName)
+      .getSchemaLastUpdatedTime(relationRaw.databaseName, relationRaw.tableName)
     val big = new CarbonScanRDD(
       ocRaw.sparkContext,
       model,
       buildCarbonPlan.getFilterExpression,
       kv,
       conf,
-      cubeCreationTime,
+      tableCreationTime,
       schemaLastUpdatedTime,
       carbonCatalog.storePath)
     big

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/84b476bc/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
index 1694a7e..361d882 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonSqlParser.scala
@@ -197,10 +197,10 @@ class CarbonSqlParser()
     (aggregation).? ~ (",".? ~> partitioner).?
   protected lazy val aggOptions =
     (noDictionaryDims).? ~ (",".? ~> aggregation).? ~ (",".? ~> partitioner).?
-  protected lazy val showcreateCubeOptionDef =
+  protected lazy val showcreateTableOptionDef =
     ("(" ~> aggOptionsForShowCreate <~ ")")
 
-  protected lazy val createCubeOptionDef =
+  protected lazy val createTableOptionDef =
     ("(" ~> aggOptions <~ ")")
 
   protected val escapedIdentifier = "`([^`]+)`".r
@@ -536,7 +536,7 @@ class CarbonSqlParser()
   Option[Partitioner] = {
 
     // by default setting partition class empty.
-    // later in cube schema it is setting to default value.
+    // later in table schema it is setting to default value.
     var partitionClass: String = ""
     var partitionCount: Int = 1
     var partitionColNames: Array[String] = Array[String]()
@@ -890,15 +890,15 @@ class CarbonSqlParser()
     LOAD ~> DATA ~> opt(LOCAL) ~> INPATH ~> stringLit ~ opt(OVERWRITE) ~
       (INTO ~> TABLE ~> (ident <~ ".").? ~ ident) ~
       (OPTIONS ~> "(" ~> repsep(loadOptions, ",") <~ ")").? <~ opt(";") ^^ {
-        case filePath ~ isOverwrite ~ cube ~ partionDataOptions =>
-          val (schema, cubename) = cube match {
-            case schemaName ~ cubeName => (schemaName, cubeName.toLowerCase())
+        case filePath ~ isOverwrite ~ table ~ partionDataOptions =>
+          val (schema, tablename) = table match {
+            case databaseName ~ tableName => (databaseName, tableName.toLowerCase())
           }
           if(partionDataOptions.isDefined) {
             validateOptions(partionDataOptions)
           }
           val patitionOptionsMap = partionDataOptions.getOrElse(List.empty[(String, String)]).toMap
-          LoadTable(schema, cubename, filePath, Seq(), patitionOptionsMap, isOverwrite.isDefined)
+          LoadTable(schema, tablename, filePath, Seq(), patitionOptionsMap, isOverwrite.isDefined)
       }
 
   private def validateOptions(partionDataOptions: Option[List[(String, String)]]): Unit = {
@@ -1161,8 +1161,8 @@ class CarbonSqlParser()
     SHOW ~> (LOADS|SEGMENTS) ~> FOR ~> TABLE ~> (ident <~ ".").? ~ ident ~
       (LIMIT ~> numericLit).? <~
       opt(";") ^^ {
-      case schemaName ~ cubeName ~ limit =>
-        ShowLoadsCommand(schemaName, cubeName.toLowerCase(), limit)
+      case databaseName ~ tableName ~ limit =>
+        ShowLoadsCommand(databaseName, tableName.toLowerCase(), limit)
     }
 
   protected lazy val segmentId: Parser[String] =
@@ -1175,8 +1175,9 @@ class CarbonSqlParser()
     DELETE ~> (LOAD|SEGMENT) ~> repsep(segmentId, ",") ~ (FROM ~> TABLE ~>
       (ident <~ ".").? ~ ident) <~
       opt(";") ^^ {
-      case loadids ~ cube => cube match {
-        case schemaName ~ cubeName => DeleteLoadsById(loadids, schemaName, cubeName.toLowerCase())
+      case loadids ~ table => table match {
+        case databaseName ~ tableName =>
+          DeleteLoadsById(loadids, databaseName, tableName.toLowerCase())
       }
     }
 
@@ -1184,16 +1185,16 @@ class CarbonSqlParser()
     DELETE ~> (LOADS|SEGMENTS) ~> FROM ~> TABLE ~> (ident <~ ".").? ~ ident ~
       (WHERE ~> (STARTTIME <~ BEFORE) ~ stringLit) <~
       opt(";") ^^ {
-      case schema ~ cube ~ condition =>
+      case schema ~ table ~ condition =>
         condition match {
           case dateField ~ dateValue =>
-            DeleteLoadsByLoadDate(schema, cube.toLowerCase(), dateField, dateValue)
+            DeleteLoadsByLoadDate(schema, table.toLowerCase(), dateField, dateValue)
         }
     }
 
   protected lazy val cleanFiles: Parser[LogicalPlan] =
     CLEAN ~> FILES ~> FOR ~> TABLE ~> (ident <~ ".").? ~ ident <~ opt(";") ^^ {
-      case schemaName ~ cubeName => CleanFiles(schemaName, cubeName.toLowerCase())
+      case databaseName ~ tableName => CleanFiles(databaseName, tableName.toLowerCase())
     }
 
 }


Mime
View raw message