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 <jacky.li...@huawei.com> Authored: Mon Nov 28 17:03:23 2016 +0800 Committer: jackylk <jacky.li...@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)