You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ja...@apache.org on 2016/11/28 12:09:59 UTC

[1/2] incubator-carbondata git commit: clean up carbonTableSchema.scala

Repository: incubator-carbondata
Updated Branches:
  refs/heads/master 86f48f51a -> 1f0955841


clean up carbonTableSchema.scala

fix

modify testcase


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

Branch: refs/heads/master
Commit: e02f021cd182441f574fd2d7b704cc1aad33d65c
Parents: 86f48f5
Author: jackylk <ja...@huawei.com>
Authored: Mon Nov 28 17:03:23 2016 +0800
Committer: jackylk <ja...@huawei.com>
Committed: Mon Nov 28 19:54:51 2016 +0800

----------------------------------------------------------------------
 .../dictionary/ReverseDictionaryCacheTest.java  |   5 +-
 .../org/apache/spark/sql/CarbonSqlParser.scala  |  16 +-
 .../execution/command/carbonTableSchema.scala   | 325 -------------------
 .../spark/sql/hive/CarbonMetastoreCatalog.scala | 123 +------
 4 files changed, 16 insertions(+), 453 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e02f021c/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
----------------------------------------------------------------------
diff --git a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
index 7f41bd6..55601c4 100644
--- a/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
+++ b/core/src/test/java/org/apache/carbondata/core/cache/dictionary/ReverseDictionaryCacheTest.java
@@ -19,7 +19,9 @@
 
 package org.apache.carbondata.core.cache.dictionary;
 
+import static org.junit.Assert.assertEquals;
 import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
 
 import java.util.ArrayList;
 import java.util.List;
@@ -177,10 +179,11 @@ public class ReverseDictionaryCacheTest extends AbstractDictionaryCacheTest {
     Dictionary reverseDictionary = null;
     try {
       reverseDictionary = (Dictionary) reverseDictionaryCache.get(dictionaryColumnUniqueIdentifier);
+      fail("not throwing exception");
     } catch (Exception e) {
       assertTrue(e instanceof CarbonUtilException);
     }
-    assertTrue(null == reverseDictionary);
+    assertEquals(null, reverseDictionary);
   }
 
   @Test public void testLRUCacheForKeyDeletionAfterMaxSizeIsReached() throws Exception {

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e02f021c/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 724ec6e..62f1d4c 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
@@ -534,13 +534,19 @@ class CarbonSqlParser() extends AbstractSparkSQLParser {
     // validate the tableBlockSize from table properties
     CommonUtil.validateTableBlockSize(tableProperties)
 
-    tableModel(ifNotExistPresent,
+    tableModel(
+      ifNotExistPresent,
       dbName.getOrElse(CarbonCommonConstants.DATABASE_DEFAULT_NAME),
-      dbName, tableName, tableProperties,
+      dbName,
+      tableName,
+      tableProperties,
       reorderDimensions(dims.map(f => normalizeType(f)).map(f => addParent(f))),
-      msrs.map(f => normalizeType(f)), "", null, "",
-      None, Seq(), null, Option(noDictionaryDims), Option(noInvertedIdxCols), null, partitioner,
-      groupCols, Some(colProps))
+      msrs.map(f => normalizeType(f)),
+      Option(noDictionaryDims),
+      Option(noInvertedIdxCols),
+      partitioner,
+      groupCols,
+      Some(colProps))
   }
 
   /**

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e02f021c/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
index 74b0dd2..22cc548 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/carbonTableSchema.scala
@@ -73,15 +73,8 @@ case class tableModel(
     tableProperties: Map[String, String],
     dimCols: Seq[Field],
     msrCols: Seq[Field],
-    fromKeyword: String,
-    withKeyword: String,
-    source: Object,
-    factFieldsList: Option[FilterCols],
-    dimRelations: Seq[DimensionRelation],
-    simpleDimRelations: Seq[DimensionRelation],
     highcardinalitydims: Option[Seq[String]],
     noInvertedIdxCols: Option[Seq[String]],
-    aggregation: Seq[Aggregation],
     partitioner: Option[Partitioner],
     columnGroups: Seq[String],
     colProps: Option[util.Map[String, util.List[ColumnProperty]]] = None)
@@ -91,61 +84,17 @@ case class Field(column: String, var dataType: Option[String], name: Option[Stri
     storeType: Option[String] = Some("columnar"),
     var precision: Int = 0, var scale: Int = 0)
 
-case class ArrayDataType(dataType: String)
-
-case class StructDataType(dataTypes: List[String])
-
-case class StructField(column: String, dataType: String)
-
-case class FieldMapping(levelName: String, columnName: String)
-
-case class HierarchyMapping(hierName: String, hierType: String, levels: Seq[String])
-
 case class ColumnProperty(key: String, value: String)
 
 case class ComplexField(complexType: String, primitiveField: Option[Field],
     complexField: Option[ComplexField])
 
-case class Cardinality(levelName: String, cardinality: Int)
-
-case class Aggregation(msrName: String, aggType: String)
-
-case class AggregateTableAttributes(colName: String, aggType: String = null)
-
 case class Partitioner(partitionClass: String, partitionColumn: Array[String], partitionCount: Int,
     nodeList: Array[String])
 
 case class PartitionerField(partitionColumn: String, dataType: Option[String],
     columnComment: String)
 
-case class DimensionRelation(tableName: String, dimSource: Object, relation: Relation,
-    includeKey: Option[String], cols: Option[Seq[String]])
-
-case class Relation(leftColumn: String, rightColumn: String)
-
-case class LoadSchema(tableInfo: TableInfo, dimensionTables: Array[DimensionRelation])
-
-case class Level(name: String, column: String, cardinality: Int, dataType: String,
-    parent: String = null, storeType: String = "Columnar",
-    levelType: String = "Regular")
-
-case class Measure(name: String, column: String, dataType: String, aggregator: String = "SUM",
-    visible: Boolean = true)
-
-case class Hierarchy(name: String, primaryKey: Option[String], levels: Seq[Level],
-    tableName: Option[String], normalized: Boolean = false)
-
-case class Dimension(name: String, hierarchies: Seq[Hierarchy], foreignKey: Option[String],
-    dimType: String = "StandardDimension", visible: Boolean = true,
-    var highCardinality: Boolean = false)
-
-case class FilterCols(includeKey: String, fieldList: Seq[String])
-
-case class Table(databaseName: String, tableName: String, dimensions: Seq[Dimension],
-    measures: Seq[Measure], partitioner: Partitioner)
-
-case class Default(key: String, value: String)
-
 case class DataLoadTableFileMapping(table: String, loadPath: String)
 
 case class CarbonMergerMapping(storeLocation: String,
@@ -166,7 +115,6 @@ case class CarbonMergerMapping(storeLocation: String,
 
 case class NodeInfo(TaskId: String, noOfBlocks: Int)
 
-
 case class AlterTableModel(dbName: Option[String], tableName: String,
     compactionType: String, alterSql: String)
 
@@ -494,279 +442,6 @@ class TableNewProcessor(cm: tableModel, sqlContext: SQLContext) {
   }
 }
 
-object TableProcessor {
-  def apply(cm: tableModel, sqlContext: SQLContext): Table = {
-    new TableProcessor(cm, sqlContext).process()
-  }
-}
-
-class TableProcessor(cm: tableModel, sqlContext: SQLContext) {
-  val timeDims = Seq("TimeYears", "TimeMonths", "TimeDays", "TimeHours", "TimeMinutes")
-  val numericTypes = Seq(CarbonCommonConstants.INTEGER_TYPE, CarbonCommonConstants.DOUBLE_TYPE,
-    CarbonCommonConstants.LONG_TYPE, CarbonCommonConstants.FLOAT_TYPE)
-
-  def getAllChildren(fieldChildren: Option[List[Field]]): Seq[Level] = {
-    var levels: Seq[Level] = Seq[Level]()
-    fieldChildren.foreach(fields => {
-      fields.foreach(field => {
-        if (field.parent != null) {
-          levels ++= Seq(Level(field.name.getOrElse(field.column), field.column, Int.MaxValue,
-            field.dataType.getOrElse(CarbonCommonConstants.STRING), field.parent,
-            field.storeType.getOrElse("Columnar")))
-        } else {
-          levels ++= Seq(Level(field.name.getOrElse(field.column), field.column, Int.MaxValue,
-            field.dataType.getOrElse(CarbonCommonConstants.STRING),
-            field.storeType.getOrElse("Columnar")))
-        }
-        if (field.children.get != null) {
-          levels ++= getAllChildren(field.children)
-        }
-      })
-    })
-    levels
-  }
-
-  def process(): Table = {
-
-    var levels = Seq[Level]()
-    var measures = Seq[Measure]()
-    var dimSrcDimensions = Seq[Dimension]()
-    val LOGGER = LogServiceFactory.getLogService(TableProcessor.getClass.getName)
-
-    // Create Table DDL with Database defination
-    cm.dimCols.foreach(field => {
-      if (field.parent != null) {
-        levels ++= Seq(Level(field.name.getOrElse(field.column), field.column, Int.MaxValue,
-          field.dataType.getOrElse(CarbonCommonConstants.STRING), field.parent,
-          field.storeType.getOrElse(CarbonCommonConstants.COLUMNAR)))
-      } else {
-        levels ++= Seq(Level(field.name.getOrElse(field.column), field.column, Int.MaxValue,
-          field.dataType.getOrElse(CarbonCommonConstants.STRING), field.parent,
-          field.storeType.getOrElse(CarbonCommonConstants.COLUMNAR)))
-      }
-      if (field.children.get != null) {
-        levels ++= getAllChildren(field.children)
-      }
-    })
-    measures = cm.msrCols.map(field => Measure(field.name.getOrElse(field.column), field.column,
-      field.dataType.getOrElse(CarbonCommonConstants.NUMERIC)))
-
-    if (cm.withKeyword.equalsIgnoreCase(CarbonCommonConstants.WITH) &&
-        cm.simpleDimRelations.nonEmpty) {
-      cm.simpleDimRelations.foreach(relationEntry => {
-
-        // Split the levels and seperate levels with dimension levels
-        val split = levels.partition(x => relationEntry.cols.get.contains(x.name))
-
-        val dimLevels = split._1
-        levels = split._2
-
-        def getMissingRelationLevel: Level = {
-          Level(relationEntry.relation.rightColumn,
-            relationEntry.relation.rightColumn, Int.MaxValue, CarbonCommonConstants.STRING)
-        }
-
-        val dimHierarchies = dimLevels.map(field =>
-          Hierarchy(relationEntry.tableName, Some(dimLevels.find(dl =>
-            dl.name.equalsIgnoreCase(relationEntry.relation.rightColumn))
-            .getOrElse(getMissingRelationLevel).column),
-            Seq(field), Some(relationEntry.tableName)))
-        dimSrcDimensions = dimSrcDimensions ++ dimHierarchies.map(
-          field => Dimension(field.levels.head.name, Seq(field),
-            Some(relationEntry.relation.leftColumn)))
-      })
-    }
-
-    // Check if there is any duplicate measures or dimensions.
-    // Its based on the dimension name and measure name
-    levels.groupBy(_.name).foreach(f => if (f._2.size > 1) {
-      val name = f._1
-      LOGGER.error(s"Duplicate dimensions found with name: $name")
-      LOGGER.audit(
-        "Validation failed for Create/Alter Table Operation " +
-        s"for ${ cm.databaseName }.${ cm.tableName } " +
-        s"Duplicate dimensions found with name: $name")
-      sys.error(s"Duplicate dimensions found with name: $name")
-    })
-
-    levels.groupBy(_.column).foreach(f => if (f._2.size > 1) {
-      val name = f._1
-      LOGGER.error(s"Duplicate dimensions found with column name: $name")
-      LOGGER.audit(
-        "Validation failed for Create/Alter Table Operation " +
-        s"for ${ cm.databaseName }.${ cm.tableName } " +
-        s"Duplicate dimensions found with column name: $name")
-      sys.error(s"Duplicate dimensions found with column name: $name")
-    })
-
-    measures.groupBy(_.name).foreach(f => if (f._2.size > 1) {
-      val name = f._1
-      LOGGER.error(s"Duplicate measures found with name: $name")
-      LOGGER.audit(
-        s"Validation failed for Create/Alter Table Operation " +
-        s"for ${ cm.databaseName }.${ cm.tableName } " +
-        s"Duplicate measures found with name: $name")
-      sys.error(s"Duplicate measures found with name: $name")
-    })
-
-    measures.groupBy(_.column).foreach(f => if (f._2.size > 1) {
-      val name = f._1
-      LOGGER.error(s"Duplicate measures found with column name: $name")
-      LOGGER.audit(
-        s"Validation failed for Create/Alter Table Operation " +
-        s"for ${ cm.databaseName }.${ cm.tableName } " +
-        s"Duplicate measures found with column name: $name")
-      sys.error(s"Duplicate measures found with column name: $name")
-    })
-
-    val levelsArray = levels.map(_.name)
-    val levelsNdMesures = levelsArray ++ measures.map(_.name)
-
-    cm.aggregation.foreach(a => {
-      if (levelsArray.contains(a.msrName)) {
-        val fault = a.msrName
-        LOGGER.error(s"Aggregator should not be defined for dimension fields [$fault]")
-        LOGGER.audit(
-          s"Validation failed for Create/Alter Table Operation for " +
-          s"${ cm.databaseName }.${ cm.tableName } " +
-          s"Aggregator should not be defined for dimension fields [$fault]")
-        sys.error(s"Aggregator should not be defined for dimension fields [$fault]")
-      }
-    })
-
-    levelsNdMesures.groupBy(x => x).foreach(f => if (f._2.size > 1) {
-      val name = f._1
-      LOGGER.error(s"Dimension and Measure defined with same name: $name")
-      LOGGER.audit(
-        s"Validation failed for Create/Alter Table Operation " +
-        s"for ${ cm.databaseName }.${ cm.tableName } " +
-        s"Dimension and Measure defined with same name: $name")
-      sys.error(s"Dimension and Measure defined with same name: $name")
-    })
-
-    dimSrcDimensions.foreach(d => {
-      d.hierarchies.foreach(h => {
-        h.levels.foreach(l => {
-          levels = levels.dropWhile(lev => lev.name.equalsIgnoreCase(l.name))
-        })
-      })
-    })
-
-    val groupedSeq = levels.groupBy(_.name.split('.')(0))
-    val hierarchies = levels.filter(level => !level.name.contains(".")).map(
-      parentLevel => Hierarchy(parentLevel.name, None, groupedSeq.get(parentLevel.name).get, None))
-    var dimensions = hierarchies.map(field => Dimension(field.name, Seq(field), None))
-
-    dimensions = dimensions ++ dimSrcDimensions
-    val highCardinalityDims = cm.highcardinalitydims.getOrElse(Seq())
-    for (dimension <- dimensions) {
-
-      if (highCardinalityDims.contains(dimension.name)) {
-        dimension.highCardinality = true
-      }
-
-    }
-
-    if (measures.length <= 0) {
-      measures = measures ++ Seq(Measure(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE,
-        CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE, CarbonCommonConstants.NUMERIC,
-        CarbonCommonConstants.SUM, visible = false))
-    }
-
-    // Update measures with aggregators if specified.
-    val msrsUpdatedWithAggregators = cm.aggregation match {
-      case aggs: Seq[Aggregation] =>
-        measures.map { f =>
-          val matchedMapping = aggs.filter(agg => f.name.equals(agg.msrName))
-          if (matchedMapping.isEmpty) {
-            f
-          } else {
-            Measure(f.name, f.column, f.dataType, matchedMapping.head.aggType)
-          }
-        }
-      case _ => measures
-    }
-
-    val partitioner = cm.partitioner match {
-      case Some(part: Partitioner) =>
-        var definedpartCols = part.partitionColumn
-        val columnBuffer = new ArrayBuffer[String]
-        part.partitionColumn.foreach { col =>
-          dimensions.foreach { dim =>
-            dim.hierarchies.foreach { hier =>
-              hier.levels.foreach { lev =>
-                if (lev.name.equalsIgnoreCase(col)) {
-                  definedpartCols = definedpartCols.dropWhile(c => c.equals(col))
-                  columnBuffer += lev.name
-                }
-              }
-            }
-          }
-        }
-
-
-        // Special Case, where Partition count alone is sent to Carbon for dataloading
-        if (part.partitionClass.isEmpty && part.partitionColumn(0).isEmpty) {
-          Partitioner(
-            "org.apache.carbondata.spark.partition.api.impl.SampleDataPartitionerImpl",
-            Array(""), part.partitionCount, null)
-        } else if (definedpartCols.nonEmpty) {
-          val msg = definedpartCols.mkString(", ")
-          LOGGER.error(s"partition columns specified are not part of Dimension columns: $msg")
-          LOGGER.audit(
-            s"Validation failed for Create/Alter Table Operation - " +
-            s"partition columns specified are not part of Dimension columns: $msg")
-          sys.error(s"partition columns specified are not part of Dimension columns: $msg")
-        } else {
-          try {
-            Class.forName(part.partitionClass).newInstance()
-          } catch {
-            case e: Exception =>
-              val cl = part.partitionClass
-              LOGGER.audit(
-                s"Validation failed for Create/Alter Table Operation for " +
-                s"${ cm.databaseName }.${ cm.tableName } " +
-                s"partition class specified can not be found or loaded: $cl")
-              sys.error(s"partition class specified can not be found or loaded: $cl")
-          }
-
-          Partitioner(part.partitionClass, columnBuffer.toArray, part.partitionCount, null)
-        }
-      case None =>
-        Partitioner("org.apache.carbondata.spark.partition.api.impl.SampleDataPartitionerImpl",
-          Array(""), 20, null)
-    }
-
-    Table(cm.databaseName, cm.tableName, dimensions, msrsUpdatedWithAggregators, partitioner)
-  }
-
-  // For filtering INCLUDE and EXCLUDE fields if any is defined for Dimention relation
-  def filterRelIncludeCols(relationEntry: DimensionRelation, p: (String, String)): Boolean = {
-    if (relationEntry.includeKey.get.equalsIgnoreCase(CarbonCommonConstants.INCLUDE)) {
-      relationEntry.cols.get.map(x => x.toLowerCase()).contains(p._1.toLowerCase())
-    } else {
-      !relationEntry.cols.get.map(x => x.toLowerCase()).contains(p._1.toLowerCase())
-    }
-  }
-
-}
-
-// These are the assumptions made
-// 1.We have a single hierarchy under a dimension tag and a single level under a hierarchy tag
-// 2.The names of dimensions and measures are case insensitive
-// 3.CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE is always added as a measure.
-// So we need to ignore this to check duplicates
-private[sql] case class AlterTable(
-    cm: tableModel,
-    dropCols: Seq[String],
-    defaultVals: Seq[Default]) extends RunnableCommand {
-
-  def run(sqlContext: SQLContext): Seq[Row] = {
-    // TODO : Implement it.
-    Seq.empty
-  }
-}
-
 /**
  * Command for the compaction in alter table command
  *

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/e02f021c/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
----------------------------------------------------------------------
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
index 3ddbb41..d219bcb 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonMetastoreCatalog.scala
@@ -30,7 +30,7 @@ import org.apache.spark.sql._
 import org.apache.spark.sql.catalyst.TableIdentifier
 import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.execution.command.{AggregateTableAttributes, Partitioner}
+import org.apache.spark.sql.execution.command.Partitioner
 import org.apache.spark.sql.hive.client.ClientInterface
 import org.apache.spark.sql.types._
 
@@ -346,47 +346,6 @@ class CarbonMetastoreCatalog(hiveContext: HiveContext, val storePath: String,
     updateMetadataByWrapperTable(wrapperTableInfo)
   }
 
-
-  def getDimensions(carbonTable: CarbonTable,
-      aggregateAttributes: List[AggregateTableAttributes]): Array[String] = {
-    var dimArray = Array[String]()
-    aggregateAttributes.filter { agg => null == agg.aggType }.foreach { agg =>
-      val colName = agg.colName
-      if (null != carbonTable.getMeasureByName(carbonTable.getFactTableName, colName)) {
-        sys
-          .error(s"Measure must be provided along with aggregate function :: $colName")
-      }
-      if (null == carbonTable.getDimensionByName(carbonTable.getFactTableName, colName)) {
-        sys
-          .error(s"Invalid column name. Cannot create an aggregate table :: $colName")
-      }
-      if (dimArray.contains(colName)) {
-        sys.error(s"Duplicate column name. Cannot create an aggregate table :: $colName")
-      }
-      dimArray :+= colName
-    }
-    dimArray
-  }
-
-  /**
-   * Shows all schemas which has Database name like
-   */
-  def showDatabases(schemaLike: Option[String]): Seq[String] = {
-    checkSchemasModifiedTimeAndReloadTables()
-    metadata.tablesMeta.map { c =>
-      schemaLike match {
-        case Some(name) =>
-          if (c.carbonTableIdentifier.getDatabaseName.contains(name)) {
-            c.carbonTableIdentifier
-              .getDatabaseName
-          } else {
-            null
-          }
-        case _ => c.carbonTableIdentifier.getDatabaseName
-      }
-    }.filter(f => f != null)
-  }
-
   /**
    * Shows all tables for given schema.
    */
@@ -400,17 +359,6 @@ class CarbonMetastoreCatalog(hiveContext: HiveContext, val storePath: String,
     }.map { c => (c.carbonTableIdentifier.getTableName, false) }
   }
 
-  /**
-   * Shows all tables in all schemas.
-   */
-  def getAllTables()(sqlContext: SQLContext): Seq[TableIdentifier] = {
-    checkSchemasModifiedTimeAndReloadTables()
-    metadata.tablesMeta.map { c =>
-      TableIdentifier(c.carbonTableIdentifier.getTableName,
-        Some(c.carbonTableIdentifier.getDatabaseName))
-    }
-  }
-
   def isTablePathExists(tableIdentifier: TableIdentifier)(sqlContext: SQLContext): Boolean = {
     val dbName = tableIdentifier.database.getOrElse(getDB.getDatabaseName(None, sqlContext))
     val tableName = tableIdentifier.table
@@ -532,75 +480,6 @@ class CarbonMetastoreCatalog(hiveContext: HiveContext, val storePath: String,
     schemaLastUpdatedTime
   }
 
-  def readTableMetaDataFile(tableFolder: CarbonFile,
-      fileType: FileFactory.FileType):
-  (String, String, String, String, Partitioner, Long) = {
-    val tableMetadataFile = tableFolder.getAbsolutePath + "/metadata"
-
-    var schema: String = ""
-    var databaseName: String = ""
-    var tableName: String = ""
-    var dataPath: String = ""
-    var partitioner: Partitioner = null
-    val cal = new GregorianCalendar(2011, 1, 1)
-    var tableCreationTime = cal.getTime.getTime
-
-    if (FileFactory.isFileExist(tableMetadataFile, fileType)) {
-      // load metadata
-      val in = FileFactory.getDataInputStream(tableMetadataFile, fileType)
-      var len = 0
-      try {
-        len = in.readInt()
-      } catch {
-        case others: EOFException => len = 0
-      }
-
-      while (len > 0) {
-        val databaseNameBytes = new Array[Byte](len)
-        in.readFully(databaseNameBytes)
-
-        databaseName = new String(databaseNameBytes, "UTF8")
-        val tableNameLen = in.readInt()
-        val tableNameBytes = new Array[Byte](tableNameLen)
-        in.readFully(tableNameBytes)
-        tableName = new String(tableNameBytes, "UTF8")
-
-        val dataPathLen = in.readInt()
-        val dataPathBytes = new Array[Byte](dataPathLen)
-        in.readFully(dataPathBytes)
-        dataPath = new String(dataPathBytes, "UTF8")
-
-        val versionLength = in.readInt()
-        val versionBytes = new Array[Byte](versionLength)
-        in.readFully(versionBytes)
-
-        val schemaLen = in.readInt()
-        val schemaBytes = new Array[Byte](schemaLen)
-        in.readFully(schemaBytes)
-        schema = new String(schemaBytes, "UTF8")
-
-        val partitionLength = in.readInt()
-        val partitionBytes = new Array[Byte](partitionLength)
-        in.readFully(partitionBytes)
-        val inStream = new ByteArrayInputStream(partitionBytes)
-        val objStream = new ObjectInputStream(inStream)
-        partitioner = objStream.readObject().asInstanceOf[Partitioner]
-        objStream.close()
-
-        try {
-          tableCreationTime = in.readLong()
-          len = in.readInt()
-        } catch {
-          case others: EOFException => len = 0
-        }
-
-      }
-      in.close()
-    }
-
-    (databaseName, tableName, dataPath, schema, partitioner, tableCreationTime)
-  }
-
   def createDatabaseDirectory(dbName: String) {
     val databasePath = storePath + File.separator + dbName
     val fileType = FileFactory.getFileType(databasePath)


[2/2] incubator-carbondata git commit: [CARBONDATA-462] Clean up carbonTableSchema.scala before moving to spark-common module This closes #360

Posted by ja...@apache.org.
[CARBONDATA-462] Clean up carbonTableSchema.scala before moving to spark-common module This closes #360


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

Branch: refs/heads/master
Commit: 1f095584146b487896278194265118c33f945179
Parents: 86f48f5 e02f021
Author: jackylk <ja...@huawei.com>
Authored: Mon Nov 28 20:08:14 2016 +0800
Committer: jackylk <ja...@huawei.com>
Committed: Mon Nov 28 20:08:14 2016 +0800

----------------------------------------------------------------------
 .../dictionary/ReverseDictionaryCacheTest.java  |   5 +-
 .../org/apache/spark/sql/CarbonSqlParser.scala  |  16 +-
 .../execution/command/carbonTableSchema.scala   | 325 -------------------
 .../spark/sql/hive/CarbonMetastoreCatalog.scala | 123 +------
 4 files changed, 16 insertions(+), 453 deletions(-)
----------------------------------------------------------------------