You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ch...@apache.org on 2016/08/15 07:09:29 UTC

[44/52] [partial] incubator-carbondata git commit: Renamed packages to org.apache.carbondata and fixed errors

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java
new file mode 100644
index 0000000..8cf929f
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/converter/ThriftWrapperSchemaConverterImpl.java
@@ -0,0 +1,386 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.metadata.converter;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.apache.carbondata.core.carbon.metadata.schema.SchemaEvolution;
+import org.apache.carbondata.core.carbon.metadata.schema.SchemaEvolutionEntry;
+import org.apache.carbondata.core.carbon.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.carbon.metadata.schema.table.TableSchema;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+
+/**
+ * Thrift schema to carbon schema converter and vice versa
+ */
+public class ThriftWrapperSchemaConverterImpl implements SchemaConverter {
+
+  /* (non-Javadoc)
+   * Converts  from wrapper to thrift schema evolution entry
+   */
+  @Override
+  public org.apache.carbondata.format.SchemaEvolutionEntry
+       fromWrapperToExternalSchemaEvolutionEntry(SchemaEvolutionEntry wrapperSchemaEvolutionEntry) {
+    org.apache.carbondata.format.SchemaEvolutionEntry thriftSchemaEvolutionEntry =
+        new org.apache.carbondata.format.SchemaEvolutionEntry(
+            wrapperSchemaEvolutionEntry.getTimeStamp());
+
+    List<org.apache.carbondata.format.ColumnSchema> thriftAddedColumns =
+        new ArrayList<org.apache.carbondata.format.ColumnSchema>();
+    for (ColumnSchema wrapperColumnSchema : wrapperSchemaEvolutionEntry.getAdded()) {
+      thriftAddedColumns.add(fromWrapperToExternalColumnSchema(wrapperColumnSchema));
+    }
+
+    List<org.apache.carbondata.format.ColumnSchema> thriftRemovedColumns =
+        new ArrayList<org.apache.carbondata.format.ColumnSchema>();
+    for (ColumnSchema wrapperColumnSchema : wrapperSchemaEvolutionEntry.getRemoved()) {
+      thriftRemovedColumns.add(fromWrapperToExternalColumnSchema(wrapperColumnSchema));
+    }
+
+    thriftSchemaEvolutionEntry.setAdded(thriftAddedColumns);
+    thriftSchemaEvolutionEntry.setRemoved(thriftRemovedColumns);
+    return thriftSchemaEvolutionEntry;
+  }
+
+  /* (non-Javadoc)
+   * converts from wrapper to thrift schema evolution
+   */
+  @Override
+  public org.apache.carbondata.format.SchemaEvolution fromWrapperToExternalSchemaEvolution(
+      SchemaEvolution wrapperSchemaEvolution) {
+
+    List<org.apache.carbondata.format.SchemaEvolutionEntry> thriftSchemaEvolEntryList =
+        new ArrayList<org.apache.carbondata.format.SchemaEvolutionEntry>();
+    for (SchemaEvolutionEntry schemaEvolutionEntry : wrapperSchemaEvolution
+        .getSchemaEvolutionEntryList()) {
+      thriftSchemaEvolEntryList
+          .add(fromWrapperToExternalSchemaEvolutionEntry(schemaEvolutionEntry));
+    }
+    return new org.apache.carbondata.format.SchemaEvolution(thriftSchemaEvolEntryList);
+  }
+
+  /**
+   * converts from wrapper to external encoding
+   *
+   * @param encoder
+   * @return
+   */
+  private org.apache.carbondata.format.Encoding fromWrapperToExternalEncoding(Encoding encoder) {
+
+    if (null == encoder) {
+      return null;
+    }
+
+    switch (encoder) {
+      case DICTIONARY:
+        return org.apache.carbondata.format.Encoding.DICTIONARY;
+      case DELTA:
+        return org.apache.carbondata.format.Encoding.DELTA;
+      case RLE:
+        return org.apache.carbondata.format.Encoding.RLE;
+      case INVERTED_INDEX:
+        return org.apache.carbondata.format.Encoding.INVERTED_INDEX;
+      case BIT_PACKED:
+        return org.apache.carbondata.format.Encoding.BIT_PACKED;
+      case DIRECT_DICTIONARY:
+        return org.apache.carbondata.format.Encoding.DIRECT_DICTIONARY;
+      default:
+        return org.apache.carbondata.format.Encoding.DICTIONARY;
+    }
+  }
+
+  /**
+   * convert from wrapper to external data type
+   *
+   * @param dataType
+   * @return
+   */
+  private org.apache.carbondata.format.DataType fromWrapperToExternalDataType(DataType dataType) {
+
+    if (null == dataType) {
+      return null;
+    }
+    switch (dataType) {
+      case STRING:
+        return org.apache.carbondata.format.DataType.STRING;
+      case INT:
+        return org.apache.carbondata.format.DataType.INT;
+      case SHORT:
+        return org.apache.carbondata.format.DataType.SHORT;
+      case LONG:
+        return org.apache.carbondata.format.DataType.LONG;
+      case DOUBLE:
+        return org.apache.carbondata.format.DataType.DOUBLE;
+      case DECIMAL:
+        return org.apache.carbondata.format.DataType.DECIMAL;
+      case TIMESTAMP:
+        return org.apache.carbondata.format.DataType.TIMESTAMP;
+      case ARRAY:
+        return org.apache.carbondata.format.DataType.ARRAY;
+      case STRUCT:
+        return org.apache.carbondata.format.DataType.STRUCT;
+      default:
+        return org.apache.carbondata.format.DataType.STRING;
+    }
+  }
+
+  /* (non-Javadoc)
+   * convert from wrapper to external column schema
+   */
+  @Override public org.apache.carbondata.format.ColumnSchema fromWrapperToExternalColumnSchema(
+      ColumnSchema wrapperColumnSchema) {
+
+    List<org.apache.carbondata.format.Encoding> encoders =
+        new ArrayList<org.apache.carbondata.format.Encoding>();
+    for (Encoding encoder : wrapperColumnSchema.getEncodingList()) {
+      encoders.add(fromWrapperToExternalEncoding(encoder));
+    }
+    org.apache.carbondata.format.ColumnSchema thriftColumnSchema =
+        new org.apache.carbondata.format.ColumnSchema(
+            fromWrapperToExternalDataType(wrapperColumnSchema.getDataType()),
+            wrapperColumnSchema.getColumnName(), wrapperColumnSchema.getColumnUniqueId(),
+            wrapperColumnSchema.isColumnar(), encoders, wrapperColumnSchema.isDimensionColumn());
+    thriftColumnSchema.setColumn_group_id(wrapperColumnSchema.getColumnGroupId());
+    thriftColumnSchema.setScale(wrapperColumnSchema.getScale());
+    thriftColumnSchema.setPrecision(wrapperColumnSchema.getPrecision());
+    thriftColumnSchema.setNum_child(wrapperColumnSchema.getNumberOfChild());
+    thriftColumnSchema.setDefault_value(wrapperColumnSchema.getDefaultValue());
+    thriftColumnSchema.setColumnProperties(wrapperColumnSchema.getColumnProperties());
+    thriftColumnSchema.setInvisible(wrapperColumnSchema.isInvisible());
+    thriftColumnSchema.setColumnReferenceId(wrapperColumnSchema.getColumnReferenceId());
+    return thriftColumnSchema;
+  }
+
+  /* (non-Javadoc)
+   * convert from wrapper to external tableschema
+   */
+  @Override public org.apache.carbondata.format.TableSchema fromWrapperToExternalTableSchema(
+      TableSchema wrapperTableSchema) {
+
+    List<org.apache.carbondata.format.ColumnSchema> thriftColumnSchema =
+        new ArrayList<org.apache.carbondata.format.ColumnSchema>();
+    for (ColumnSchema wrapperColumnSchema : wrapperTableSchema.getListOfColumns()) {
+      thriftColumnSchema.add(fromWrapperToExternalColumnSchema(wrapperColumnSchema));
+    }
+    org.apache.carbondata.format.SchemaEvolution schemaEvolution =
+        fromWrapperToExternalSchemaEvolution(wrapperTableSchema.getSchemaEvalution());
+    return new org.apache.carbondata.format.TableSchema(wrapperTableSchema.getTableId(),
+        thriftColumnSchema, schemaEvolution);
+  }
+
+  /* (non-Javadoc)
+   * convert from wrapper to external tableinfo
+   */
+  @Override public org.apache.carbondata.format.TableInfo fromWrapperToExternalTableInfo(
+      TableInfo wrapperTableInfo, String dbName, String tableName) {
+
+    org.apache.carbondata.format.TableSchema thriftFactTable =
+        fromWrapperToExternalTableSchema(wrapperTableInfo.getFactTable());
+    List<org.apache.carbondata.format.TableSchema> thriftAggTables =
+        new ArrayList<org.apache.carbondata.format.TableSchema>();
+    for (TableSchema wrapperAggTableSchema : wrapperTableInfo.getAggregateTableList()) {
+      thriftAggTables.add(fromWrapperToExternalTableSchema(wrapperAggTableSchema));
+    }
+    return new org.apache.carbondata.format.TableInfo(thriftFactTable, thriftAggTables);
+  }
+
+  /* (non-Javadoc)
+   * convert from external to wrapper schema evolution entry
+   */
+  @Override public SchemaEvolutionEntry fromExternalToWrapperSchemaEvolutionEntry(
+      org.apache.carbondata.format.SchemaEvolutionEntry externalSchemaEvolutionEntry) {
+
+    SchemaEvolutionEntry wrapperSchemaEvolutionEntry = new SchemaEvolutionEntry();
+    wrapperSchemaEvolutionEntry.setTimeStamp(externalSchemaEvolutionEntry.getTime_stamp());
+
+    List<ColumnSchema> wrapperAddedColumns = new ArrayList<ColumnSchema>();
+    if (null != externalSchemaEvolutionEntry.getAdded()) {
+      for (org.apache.carbondata.format.ColumnSchema externalColumnSchema :
+          externalSchemaEvolutionEntry.getAdded()) {
+        wrapperAddedColumns.add(fromExternalToWrapperColumnSchema(externalColumnSchema));
+      }
+    }
+    List<ColumnSchema> wrapperRemovedColumns = new ArrayList<ColumnSchema>();
+    if (null != externalSchemaEvolutionEntry.getRemoved()) {
+      for (org.apache.carbondata.format.ColumnSchema externalColumnSchema :
+          externalSchemaEvolutionEntry.getRemoved()) {
+        wrapperRemovedColumns.add(fromExternalToWrapperColumnSchema(externalColumnSchema));
+      }
+    }
+
+    wrapperSchemaEvolutionEntry.setAdded(wrapperAddedColumns);
+    wrapperSchemaEvolutionEntry.setRemoved(wrapperRemovedColumns);
+    return wrapperSchemaEvolutionEntry;
+
+  }
+
+  /* (non-Javadoc)
+   * convert from external to wrapper schema evolution
+   */
+  @Override public SchemaEvolution fromExternalToWrapperSchemaEvolution(
+      org.apache.carbondata.format.SchemaEvolution externalSchemaEvolution) {
+    List<SchemaEvolutionEntry> wrapperSchemaEvolEntryList = new ArrayList<SchemaEvolutionEntry>();
+    for (org.apache.carbondata.format.SchemaEvolutionEntry schemaEvolutionEntry :
+        externalSchemaEvolution.getSchema_evolution_history()) {
+      wrapperSchemaEvolEntryList
+          .add(fromExternalToWrapperSchemaEvolutionEntry(schemaEvolutionEntry));
+    }
+    SchemaEvolution wrapperSchemaEvolution = new SchemaEvolution();
+    wrapperSchemaEvolution.setSchemaEvolutionEntryList(wrapperSchemaEvolEntryList);
+    return wrapperSchemaEvolution;
+  }
+
+  /**
+   * convert from external to wrapper encoding
+   *
+   * @param encoder
+   * @return
+   */
+  private Encoding fromExternalToWrapperEncoding(org.apache.carbondata.format.Encoding encoder) {
+    if (null == encoder) {
+      return null;
+    }
+    switch (encoder) {
+      case DICTIONARY:
+        return Encoding.DICTIONARY;
+      case DELTA:
+        return Encoding.DELTA;
+      case RLE:
+        return Encoding.RLE;
+      case INVERTED_INDEX:
+        return Encoding.INVERTED_INDEX;
+      case BIT_PACKED:
+        return Encoding.BIT_PACKED;
+      case DIRECT_DICTIONARY:
+        return Encoding.DIRECT_DICTIONARY;
+      default:
+        return Encoding.DICTIONARY;
+    }
+  }
+
+  /**
+   * convert from external to wrapper data type
+   *
+   * @param dataType
+   * @return
+   */
+  private DataType fromExternalToWrapperDataType(org.apache.carbondata.format.DataType dataType) {
+    if (null == dataType) {
+      return null;
+    }
+    switch (dataType) {
+      case STRING:
+        return DataType.STRING;
+      case INT:
+        return DataType.INT;
+      case SHORT:
+        return DataType.SHORT;
+      case LONG:
+        return DataType.LONG;
+      case DOUBLE:
+        return DataType.DOUBLE;
+      case DECIMAL:
+        return DataType.DECIMAL;
+      case TIMESTAMP:
+        return DataType.TIMESTAMP;
+      case ARRAY:
+        return DataType.ARRAY;
+      case STRUCT:
+        return DataType.STRUCT;
+      default:
+        return DataType.STRING;
+    }
+  }
+
+  /* (non-Javadoc)
+   * convert from external to wrapper columnschema
+   */
+  @Override public ColumnSchema fromExternalToWrapperColumnSchema(
+      org.apache.carbondata.format.ColumnSchema externalColumnSchema) {
+    ColumnSchema wrapperColumnSchema = new ColumnSchema();
+    wrapperColumnSchema.setColumnUniqueId(externalColumnSchema.getColumn_id());
+    wrapperColumnSchema.setColumnName(externalColumnSchema.getColumn_name());
+    wrapperColumnSchema.setColumnar(externalColumnSchema.isColumnar());
+    wrapperColumnSchema.setDataType(fromExternalToWrapperDataType(externalColumnSchema.data_type));
+    wrapperColumnSchema.setDimensionColumn(externalColumnSchema.isDimension());
+    List<Encoding> encoders = new ArrayList<Encoding>();
+    for (org.apache.carbondata.format.Encoding encoder : externalColumnSchema.getEncoders()) {
+      encoders.add(fromExternalToWrapperEncoding(encoder));
+    }
+    wrapperColumnSchema.setEncodingList(encoders);
+    wrapperColumnSchema.setNumberOfChild(externalColumnSchema.getNum_child());
+    wrapperColumnSchema.setPrecision(externalColumnSchema.getPrecision());
+    wrapperColumnSchema.setColumnGroup(externalColumnSchema.getColumn_group_id());
+    wrapperColumnSchema.setScale(externalColumnSchema.getScale());
+    wrapperColumnSchema.setDefaultValue(externalColumnSchema.getDefault_value());
+    wrapperColumnSchema.setAggregateFunction(externalColumnSchema.getAggregate_function());
+    wrapperColumnSchema.setColumnProperties(externalColumnSchema.getColumnProperties());
+    wrapperColumnSchema.setInvisible(externalColumnSchema.isInvisible());
+    wrapperColumnSchema.setColumnReferenceId(externalColumnSchema.getColumnReferenceId());
+    return wrapperColumnSchema;
+  }
+
+  /* (non-Javadoc)
+   * convert from external to wrapper tableschema
+   */
+  @Override public TableSchema fromExternalToWrapperTableSchema(
+      org.apache.carbondata.format.TableSchema externalTableSchema, String tableName) {
+    TableSchema wrapperTableSchema = new TableSchema();
+    wrapperTableSchema.setTableId(externalTableSchema.getTable_id());
+    wrapperTableSchema.setTableName(tableName);
+    List<ColumnSchema> listOfColumns = new ArrayList<ColumnSchema>();
+    for (org.apache.carbondata.format.ColumnSchema externalColumnSchema : externalTableSchema
+        .getTable_columns()) {
+      listOfColumns.add(fromExternalToWrapperColumnSchema(externalColumnSchema));
+    }
+    wrapperTableSchema.setListOfColumns(listOfColumns);
+    wrapperTableSchema.setSchemaEvalution(
+        fromExternalToWrapperSchemaEvolution(externalTableSchema.getSchema_evolution()));
+    return wrapperTableSchema;
+  }
+
+  /* (non-Javadoc)
+   * convert from external to wrapper tableinfo
+   */
+  @Override public TableInfo fromExternalToWrapperTableInfo(
+      org.apache.carbondata.format.TableInfo externalTableInfo, String dbName, String tableName,
+      String storePath) {
+    TableInfo wrapperTableInfo = new TableInfo();
+    wrapperTableInfo.setLastUpdatedTime(
+        externalTableInfo.getFact_table().getSchema_evolution().getSchema_evolution_history().get(0)
+            .getTime_stamp());
+    wrapperTableInfo.setDatabaseName(dbName);
+    wrapperTableInfo.setTableUniqueName(dbName + "_" + tableName);
+    wrapperTableInfo.setStorePath(storePath);
+    wrapperTableInfo.setFactTable(
+        fromExternalToWrapperTableSchema(externalTableInfo.getFact_table(), tableName));
+    List<TableSchema> aggTablesList = new ArrayList<TableSchema>();
+    int index = 0;
+    for (org.apache.carbondata.format.TableSchema aggTable : externalTableInfo
+        .getAggregate_table_list()) {
+      aggTablesList.add(fromExternalToWrapperTableSchema(aggTable, "agg_table_" + index));
+      index++;
+    }
+    return wrapperTableInfo;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/datatype/ConvertedType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/datatype/ConvertedType.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/datatype/ConvertedType.java
new file mode 100644
index 0000000..7b99ec2
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/datatype/ConvertedType.java
@@ -0,0 +1,122 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.metadata.datatype;
+
+public enum ConvertedType {
+
+  /**
+   * a BYTE_ARRAY actually contains UTF8 encoded chars
+   */
+  UTF8,
+  /**
+   * a map is converted as an optional field containing a repeated key/value pair
+   */
+  MAP,
+  /**
+   * a key/value pair is converted into a group of two fields
+   */
+  MAP_KEY_VALUE,
+  /**
+   * a list is converted into an optional field containing a repeated field for its
+   * values
+   */
+  LIST,
+  /**
+   * an enum is converted into a binary field
+   */
+  ENUM,
+  /**
+   * A decimal value.
+   * This may be used to annotate binary or fixed primitive types. The
+   * underlying byte array stores the unscaled value encoded as two's
+   * complement using big-endian byte order (the most significant byte is the
+   * zeroth element). The value of the decimal is the value * 10^{-scale}.
+   * This must be accompanied by a (maximum) precision and a scale in the
+   * SchemaElement. The precision specifies the number of digits in the decimal
+   * and the scale stores the location of the decimal point. For example 1.23
+   * would have precision 3 (3 total digits) and scale 2 (the decimal point is
+   * 2 digits over).
+   */
+  DECIMAL,
+  /**
+   * A Date
+   * Stored as days since Unix epoch, encoded as the INT32 physical type.
+   */
+  DATE,
+  /**
+   * A time
+   * The total number of milliseconds since midnight.  The value is stored
+   * as an INT32 physical type.
+   */
+  TIME_MILLIS,
+  /**
+   * A date/time combination
+   * Date and time recorded as milliseconds since the Unix epoch.  Recorded as
+   * a physical type of INT64.
+   */
+  TIMESTAMP_MILLIS,
+
+  RESERVED,
+  /**
+   * An unsigned integer value.
+   * The number describes the maximum number of meainful data bits in
+   * the stored value. 8, 16 and 32 bit values are stored using the
+   * INT32 physical type.  64 bit values are stored using the INT64
+   * physical type.
+   */
+  UINT_8,
+  UINT_16,
+  UINT_32,
+  UINT_64,
+  /**
+   * A signed integer value.
+   * The number describes the maximum number of meainful data bits in
+   * the stored value. 8, 16 and 32 bit values are stored using the
+   * INT32 physical type.  64 bit values are stored using the INT64
+   * physical type.
+   */
+  INT_8,
+  INT_16,
+  INT_32,
+  INT_64,
+  /**
+   * An embedded JSON document
+   * A JSON document embedded within a single UTF8 column.
+   */
+  JSON,
+
+  /**
+   * An embedded BSON document
+   * A BSON document embedded within a single BINARY column.
+   */
+  BSON,
+
+  /**
+   * An interval of time
+   * This type annotates data stored as a FIXED_LEN_BYTE_ARRAY of length 12
+   * This data is composed of three separate little endian unsigned
+   * integers.  Each stores a component of a duration of time.  The first
+   * integer identifies the number of months associated with the duration,
+   * the second identifies the number of days associated with the duration
+   * and the third identifies the number of milliseconds associated with
+   * the provided duration.  This duration of time is independent of any
+   * particular timezone or date.
+   */
+  INTERVAL;
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/datatype/DataType.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/datatype/DataType.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/datatype/DataType.java
new file mode 100644
index 0000000..ac7ef32
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/datatype/DataType.java
@@ -0,0 +1,48 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.carbondata.core.carbon.metadata.datatype;
+
+public enum DataType {
+
+  STRING(0),
+  DATE(1),
+  TIMESTAMP(2),
+  BOOLEAN(1),
+  SHORT(2),
+  INT(3),
+  FLOAT(4),
+  LONG(5),
+  DOUBLE(6),
+  NULL(7),
+  DECIMAL(8),
+  ARRAY(9),
+  STRUCT(10),
+  MAP(11);
+
+  private int presedenceOrder;
+
+  DataType(int value) {
+    this.presedenceOrder = value;
+  }
+
+  public int getPresedenceOrder() {
+    return presedenceOrder;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/encoder/Encoding.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/encoder/Encoding.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/encoder/Encoding.java
new file mode 100644
index 0000000..416f8e7
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/encoder/Encoding.java
@@ -0,0 +1,31 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.metadata.encoder;
+
+/**
+ * Encoding type supported in carbon
+ */
+public enum Encoding {
+  DICTIONARY,
+  DELTA,
+  RLE,
+  INVERTED_INDEX,
+  BIT_PACKED,
+  DIRECT_DICTIONARY;
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/index/BlockIndexInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/index/BlockIndexInfo.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/index/BlockIndexInfo.java
new file mode 100644
index 0000000..9bdbc18
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/index/BlockIndexInfo.java
@@ -0,0 +1,92 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.metadata.index;
+
+import org.apache.carbondata.core.carbon.metadata.blocklet.index.BlockletIndex;
+
+/**
+ * Below class will be used hold the information
+ * about block index
+ */
+public class BlockIndexInfo {
+
+  /**
+   * total number of rows present in the file
+   */
+  private long numberOfRows;
+
+  /**
+   * file name
+   */
+  private String fileName;
+
+  /**
+   * offset of metadata in data file
+   */
+  private long offset;
+
+  /**
+   * to store min max and start and end key
+   */
+  private BlockletIndex blockletIndex;
+
+  /**
+   * Constructor
+   *
+   * @param numberOfRows  number of rows
+   * @param fileName      full qualified name
+   * @param offset        offset of the metadata in data file
+   * @param blockletIndex block let index
+   */
+  public BlockIndexInfo(long numberOfRows, String fileName, long offset,
+      BlockletIndex blockletIndex) {
+    this.numberOfRows = numberOfRows;
+    this.fileName = fileName;
+    this.offset = offset;
+    this.blockletIndex = blockletIndex;
+  }
+
+  /**
+   * @return the numberOfRows
+   */
+  public long getNumberOfRows() {
+    return numberOfRows;
+  }
+
+  /**
+   * @return the fileName
+   */
+  public String getFileName() {
+    return fileName;
+  }
+
+  /**
+   * @return the offset
+   */
+  public long getOffset() {
+    return offset;
+  }
+
+  /**
+   * @return the blockletIndex
+   */
+  public BlockletIndex getBlockletIndex() {
+    return blockletIndex;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/SchemaEvolution.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/SchemaEvolution.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/SchemaEvolution.java
new file mode 100644
index 0000000..9bbcce3
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/SchemaEvolution.java
@@ -0,0 +1,52 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.metadata.schema;
+
+import java.io.Serializable;
+import java.util.List;
+
+/**
+ * Persisting schema restructuring information;
+ */
+public class SchemaEvolution implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = 8186224567517679868L;
+
+  /**
+   * list of schema evolution entry
+   */
+  private List<SchemaEvolutionEntry> schemaEvolutionEntryList;
+
+  /**
+   * @return the schemaEvolutionEntryList
+   */
+  public List<SchemaEvolutionEntry> getSchemaEvolutionEntryList() {
+    return schemaEvolutionEntryList;
+  }
+
+  /**
+   * @param schemaEvolutionEntryList the schemaEvolutionEntryList to set
+   */
+  public void setSchemaEvolutionEntryList(List<SchemaEvolutionEntry> schemaEvolutionEntryList) {
+    this.schemaEvolutionEntryList = schemaEvolutionEntryList;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/SchemaEvolutionEntry.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/SchemaEvolutionEntry.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/SchemaEvolutionEntry.java
new file mode 100644
index 0000000..ec5fb96
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/SchemaEvolutionEntry.java
@@ -0,0 +1,93 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.metadata.schema;
+
+import java.io.Serializable;
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+
+/**
+ * Store the infomation about the schema evolution
+ */
+public class SchemaEvolutionEntry implements Serializable {
+
+  /**
+   * serilization version
+   */
+  private static final long serialVersionUID = -7619477063676325276L;
+
+  /**
+   * time stamp of restructuring
+   */
+  private long timeStamp;
+
+  /**
+   * new column added in restructuring
+   */
+  private List<ColumnSchema> added;
+
+  /**
+   * column removed in restructuring
+   */
+  private List<ColumnSchema> removed;
+
+  /**
+   * @return the timeStamp
+   */
+  public long getTimeStamp() {
+    return timeStamp;
+  }
+
+  /**
+   * @param timeStamp the timeStamp to set
+   */
+  public void setTimeStamp(long timeStamp) {
+    this.timeStamp = timeStamp;
+  }
+
+  /**
+   * @return the added
+   */
+  public List<ColumnSchema> getAdded() {
+    return added;
+  }
+
+  /**
+   * @param added the added to set
+   */
+  public void setAdded(List<ColumnSchema> added) {
+    this.added = added;
+  }
+
+  /**
+   * @return the removed
+   */
+  public List<ColumnSchema> getRemoved() {
+    return removed;
+  }
+
+  /**
+   * @param removed the removed to set
+   */
+  public void setRemoved(List<ColumnSchema> removed) {
+    this.removed = removed;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/CarbonTable.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/CarbonTable.java
new file mode 100644
index 0000000..b99b8d3
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/CarbonTable.java
@@ -0,0 +1,393 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.carbondata.core.carbon.metadata.schema.table;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.carbon.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.carbon.CarbonTableIdentifier;
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonDimension;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.CarbonMeasure;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+
+/**
+ * Mapping class for Carbon actual table
+ */
+public class CarbonTable implements Serializable {
+
+  /**
+   * serialization id
+   */
+  private static final long serialVersionUID = 8696507171227156445L;
+
+  /**
+   * Absolute table identifier
+   */
+  private AbsoluteTableIdentifier absoluteTableIdentifier;
+
+  /**
+   * TableName, Dimensions list
+   */
+  private Map<String, List<CarbonDimension>> tableDimensionsMap;
+
+  /**
+   * table measures list.
+   */
+  private Map<String, List<CarbonMeasure>> tableMeasuresMap;
+
+  /**
+   * tableUniqueName
+   */
+  private String tableUniqueName;
+
+  /**
+   * Aggregate tables name
+   */
+  private List<String> aggregateTablesName;
+
+  /**
+   * metadata file path (check if it is really required )
+   */
+  private String metaDataFilepath;
+
+  /**
+   * last updated time
+   */
+  private long tableLastUpdatedTime;
+
+  public CarbonTable() {
+    this.tableDimensionsMap = new HashMap<String, List<CarbonDimension>>();
+    this.tableMeasuresMap = new HashMap<String, List<CarbonMeasure>>();
+    this.aggregateTablesName = new ArrayList<String>();
+  }
+
+  /**
+   * @param tableInfo
+   */
+  public void loadCarbonTable(TableInfo tableInfo) {
+    this.tableLastUpdatedTime = tableInfo.getLastUpdatedTime();
+    this.tableUniqueName = tableInfo.getTableUniqueName();
+    this.metaDataFilepath = tableInfo.getMetaDataFilepath();
+    //setting unique table identifier
+    CarbonTableIdentifier carbontableIdentifier =
+        new CarbonTableIdentifier(tableInfo.getDatabaseName(),
+            tableInfo.getFactTable().getTableName(), tableInfo.getFactTable().getTableId());
+    this.absoluteTableIdentifier =
+        new AbsoluteTableIdentifier(tableInfo.getStorePath(), carbontableIdentifier);
+
+    fillDimensionsAndMeasuresForTables(tableInfo.getFactTable());
+    List<TableSchema> aggregateTableList = tableInfo.getAggregateTableList();
+    for (TableSchema aggTable : aggregateTableList) {
+      this.aggregateTablesName.add(aggTable.getTableName());
+      fillDimensionsAndMeasuresForTables(aggTable);
+    }
+  }
+
+  /**
+   * Fill dimensions and measures for carbon table
+   *
+   * @param tableSchema
+   */
+  private void fillDimensionsAndMeasuresForTables(TableSchema tableSchema) {
+    List<CarbonDimension> dimensions = new ArrayList<CarbonDimension>();
+    List<CarbonMeasure> measures = new ArrayList<CarbonMeasure>();
+    this.tableDimensionsMap.put(tableSchema.getTableName(), dimensions);
+    this.tableMeasuresMap.put(tableSchema.getTableName(), measures);
+    int dimensionOrdinal = 0;
+    int measureOrdinal = 0;
+    int keyOrdinal = 0;
+    int columnGroupOrdinal = -1;
+    int previousColumnGroupId = -1;
+    List<ColumnSchema> listOfColumns = tableSchema.getListOfColumns();
+    int complexTypeOrdinal = -1;
+    for (int i = 0; i < listOfColumns.size(); i++) {
+      ColumnSchema columnSchema = listOfColumns.get(i);
+      if (columnSchema.isDimensionColumn()) {
+        if (columnSchema.getNumberOfChild() > 0) {
+          CarbonDimension complexDimension =
+              new CarbonDimension(columnSchema, dimensionOrdinal++, -1, -1, ++complexTypeOrdinal);
+          complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
+          dimensions.add(complexDimension);
+          dimensionOrdinal =
+              readAllComplexTypeChildrens(dimensionOrdinal, columnSchema.getNumberOfChild(),
+                  listOfColumns, complexDimension);
+          i = dimensionOrdinal - 1;
+          complexTypeOrdinal = assignComplexOrdinal(complexDimension, complexTypeOrdinal);
+        } else {
+          if (!columnSchema.getEncodingList().contains(Encoding.DICTIONARY)) {
+            dimensions.add(new CarbonDimension(columnSchema, dimensionOrdinal++, -1, -1, -1));
+          } else if (columnSchema.getEncodingList().contains(Encoding.DICTIONARY)
+              && columnSchema.getColumnGroupId() == -1) {
+            dimensions
+                .add(new CarbonDimension(columnSchema, dimensionOrdinal++, keyOrdinal++, -1, -1));
+          } else {
+            columnGroupOrdinal =
+                previousColumnGroupId == columnSchema.getColumnGroupId() ? ++columnGroupOrdinal : 0;
+            previousColumnGroupId = columnSchema.getColumnGroupId();
+            dimensions.add(new CarbonDimension(columnSchema, dimensionOrdinal++, keyOrdinal++,
+                columnGroupOrdinal, -1));
+
+          }
+        }
+      } else {
+        measures.add(new CarbonMeasure(columnSchema, measureOrdinal++));
+      }
+    }
+  }
+
+  /**
+   * Read all primitive/complex children and set it as list of child carbon dimension to parent
+   * dimension
+   *
+   * @param dimensionOrdinal
+   * @param childCount
+   * @param listOfColumns
+   * @param parentDimension
+   * @return
+   */
+  private int readAllComplexTypeChildrens(int dimensionOrdinal, int childCount,
+      List<ColumnSchema> listOfColumns, CarbonDimension parentDimension) {
+    for (int i = 0; i < childCount; i++) {
+      ColumnSchema columnSchema = listOfColumns.get(dimensionOrdinal);
+      if (columnSchema.isDimensionColumn()) {
+        if (columnSchema.getNumberOfChild() > 0) {
+          CarbonDimension complexDimension =
+              new CarbonDimension(columnSchema, dimensionOrdinal++, -1, -1, -1);
+          complexDimension.initializeChildDimensionsList(columnSchema.getNumberOfChild());
+          parentDimension.getListOfChildDimensions().add(complexDimension);
+          dimensionOrdinal =
+              readAllComplexTypeChildrens(dimensionOrdinal, columnSchema.getNumberOfChild(),
+                  listOfColumns, complexDimension);
+        } else {
+          parentDimension.getListOfChildDimensions()
+              .add(new CarbonDimension(columnSchema, dimensionOrdinal++, -1, -1, -1));
+        }
+      }
+    }
+    return dimensionOrdinal;
+  }
+
+  /**
+   * Read all primitive/complex children and set it as list of child carbon dimension to parent
+   * dimension
+   */
+  private int assignComplexOrdinal(CarbonDimension parentDimension, int complexDimensionOrdianl) {
+    for (int i = 0; i < parentDimension.getNumberOfChild(); i++) {
+      CarbonDimension dimension = parentDimension.getListOfChildDimensions().get(i);
+      if (dimension.getNumberOfChild() > 0) {
+        dimension.setComplexTypeOridnal(++complexDimensionOrdianl);
+        complexDimensionOrdianl = assignComplexOrdinal(dimension, complexDimensionOrdianl);
+      } else {
+        parentDimension.getListOfChildDimensions().get(i)
+            .setComplexTypeOridnal(++complexDimensionOrdianl);
+      }
+    }
+    return complexDimensionOrdianl;
+  }
+
+  /**
+   * @return the databaseName
+   */
+  public String getDatabaseName() {
+    return absoluteTableIdentifier.getCarbonTableIdentifier().getDatabaseName();
+  }
+
+  /**
+   * @return the tabelName
+   */
+  public String getFactTableName() {
+    return absoluteTableIdentifier.getCarbonTableIdentifier().getTableName();
+  }
+
+  /**
+   * @return the tableUniqueName
+   */
+  public String getTableUniqueName() {
+    return tableUniqueName;
+  }
+
+  /**
+   * @return the metaDataFilepath
+   */
+  public String getMetaDataFilepath() {
+    return metaDataFilepath;
+  }
+
+  /**
+   * @return storepath
+   */
+  public String getStorePath() {
+    return absoluteTableIdentifier.getStorePath();
+  }
+
+  /**
+   * @return list of aggregate TablesName
+   */
+  public List<String> getAggregateTablesName() {
+    return aggregateTablesName;
+  }
+
+  /**
+   * @return the tableLastUpdatedTime
+   */
+  public long getTableLastUpdatedTime() {
+    return tableLastUpdatedTime;
+  }
+
+  /**
+   * to get the number of dimension present in the table
+   *
+   * @param tableName
+   * @return number of dimension present the table
+   */
+  public int getNumberOfDimensions(String tableName) {
+    return tableDimensionsMap.get(tableName).size();
+  }
+
+  /**
+   * to get the number of measures present in the table
+   *
+   * @param tableName
+   * @return number of measures present the table
+   */
+  public int getNumberOfMeasures(String tableName) {
+    return tableMeasuresMap.get(tableName).size();
+  }
+
+  /**
+   * to get the all dimension of a table
+   *
+   * @param tableName
+   * @return all dimension of a table
+   */
+  public List<CarbonDimension> getDimensionByTableName(String tableName) {
+    return tableDimensionsMap.get(tableName);
+  }
+
+  /**
+   * to get the all measure of a table
+   *
+   * @param tableName
+   * @return all measure of a table
+   */
+  public List<CarbonMeasure> getMeasureByTableName(String tableName) {
+    return tableMeasuresMap.get(tableName);
+  }
+
+  /**
+   * to get particular measure from a table
+   *
+   * @param tableName
+   * @param columnName
+   * @return
+   */
+  public CarbonMeasure getMeasureByName(String tableName, String columnName) {
+    List<CarbonMeasure> measureList = tableMeasuresMap.get(tableName);
+    for (CarbonMeasure measure : measureList) {
+      if (measure.getColName().equalsIgnoreCase(columnName)) {
+        return measure;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * to get particular dimension from a table
+   *
+   * @param tableName
+   * @param columnName
+   * @return
+   */
+  public CarbonDimension getDimensionByName(String tableName, String columnName) {
+    List<CarbonDimension> dimList = tableDimensionsMap.get(tableName);
+    for (CarbonDimension dim : dimList) {
+      if (dim.getColName().equalsIgnoreCase(columnName)) {
+        return dim;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * gets all children dimension for complex type
+   *
+   * @param dimName
+   * @return list of child dimensions
+   */
+  public List<CarbonDimension> getChildren(String dimName) {
+    for (List<CarbonDimension> list : tableDimensionsMap.values()) {
+      List<CarbonDimension> childDims = getChildren(dimName, list);
+      if (childDims != null) {
+        return childDims;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * returns level 2 or more child dimensions
+   *
+   * @param dimName
+   * @param dimensions
+   * @return list of child dimensions
+   */
+  public List<CarbonDimension> getChildren(String dimName, List<CarbonDimension> dimensions) {
+    for (CarbonDimension carbonDimension : dimensions) {
+      if (carbonDimension.getColName().equals(dimName)) {
+        return carbonDimension.getListOfChildDimensions();
+      } else if (null != carbonDimension.getListOfChildDimensions()
+          && carbonDimension.getListOfChildDimensions().size() > 0) {
+        List<CarbonDimension> childDims =
+            getChildren(dimName, carbonDimension.getListOfChildDimensions());
+        if (childDims != null) {
+          return childDims;
+        }
+      }
+    }
+    return null;
+  }
+
+  /**
+   * @return absolute table identifier
+   */
+  public AbsoluteTableIdentifier getAbsoluteTableIdentifier() {
+    return absoluteTableIdentifier;
+  }
+
+  /**
+   * @return carbon table identifier
+   */
+  public CarbonTableIdentifier getCarbonTableIdentifier() {
+    return absoluteTableIdentifier.getCarbonTableIdentifier();
+  }
+
+  /**
+   * gets partition count for this table
+   * TODO: to be implemented while supporting partitioning
+   */
+  public int getPartitionCount() {
+    return 1;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/TableInfo.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/TableInfo.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/TableInfo.java
new file mode 100644
index 0000000..888b898
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/TableInfo.java
@@ -0,0 +1,239 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.metadata.schema.table;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+
+/**
+ * Store the information about the table.
+ * it stores the fact table as well as aggregate table present in the schema
+ */
+public class TableInfo implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = -5034287968314105193L;
+
+  /**
+   * name of the database;
+   */
+  private String databaseName;
+
+  /**
+   * table name to group fact table and aggregate table
+   */
+  private String tableUniqueName;
+
+  /**
+   * fact table information
+   */
+  private TableSchema factTable;
+
+  /**
+   * list of aggregate table
+   */
+  private List<TableSchema> aggregateTableList;
+
+  /**
+   * last updated time to update the table if any changes
+   */
+  private long lastUpdatedTime;
+
+  /**
+   * metadata file path (check if it is really required )
+   */
+  private String metaDataFilepath;
+
+  /**
+   * store location
+   */
+  private String storePath;
+
+  public TableInfo() {
+    aggregateTableList = new ArrayList<TableSchema>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  /**
+   * @return the factTable
+   */
+  public TableSchema getFactTable() {
+    return factTable;
+  }
+
+  /**
+   * @param factTable the factTable to set
+   */
+  public void setFactTable(TableSchema factTable) {
+    this.factTable = factTable;
+  }
+
+  /**
+   * @return the aggregateTableList
+   */
+  public List<TableSchema> getAggregateTableList() {
+    return aggregateTableList;
+  }
+
+  /**
+   * @param aggregateTableList the aggregateTableList to set
+   */
+  public void setAggregateTableList(List<TableSchema> aggregateTableList) {
+    this.aggregateTableList = aggregateTableList;
+  }
+
+  /**
+   * @return the databaseName
+   */
+  public String getDatabaseName() {
+    return databaseName;
+  }
+
+  /**
+   * @param databaseName the databaseName to set
+   */
+  public void setDatabaseName(String databaseName) {
+    this.databaseName = databaseName;
+  }
+
+  public TableSchema getTableSchemaByName(String tableName) {
+    if (factTable.getTableName().equalsIgnoreCase(tableName)) {
+      return factTable;
+    }
+    for (TableSchema aggregatTableSchema : aggregateTableList) {
+      if (aggregatTableSchema.getTableName().equals(tableName)) {
+        return aggregatTableSchema;
+      }
+    }
+    return null;
+  }
+
+  public TableSchema getTableSchemaByTableId(String tableId) {
+    if (factTable.getTableId().equals(tableId)) {
+      return factTable;
+    }
+    for (TableSchema aggregatTableSchema : aggregateTableList) {
+      if (aggregatTableSchema.getTableId().equals(tableId)) {
+        return aggregatTableSchema;
+      }
+    }
+    return null;
+  }
+
+  public int getNumberOfAggregateTables() {
+    return aggregateTableList.size();
+  }
+
+  /**
+   * @return the tableUniqueName
+   */
+  public String getTableUniqueName() {
+    return tableUniqueName;
+  }
+
+  /**
+   * @param tableUniqueName the tableUniqueName to set
+   */
+  public void setTableUniqueName(String tableUniqueName) {
+    this.tableUniqueName = tableUniqueName;
+  }
+
+  /**
+   * @return the lastUpdatedTime
+   */
+  public long getLastUpdatedTime() {
+    return lastUpdatedTime;
+  }
+
+  /**
+   * @param lastUpdatedTime the lastUpdatedTime to set
+   */
+  public void setLastUpdatedTime(long lastUpdatedTime) {
+    this.lastUpdatedTime = lastUpdatedTime;
+  }
+
+  /**
+   * @return
+   */
+  public String getMetaDataFilepath() {
+    return metaDataFilepath;
+  }
+
+  /**
+   * @param metaDataFilepath
+   */
+  public void setMetaDataFilepath(String metaDataFilepath) {
+    this.metaDataFilepath = metaDataFilepath;
+  }
+
+  public String getStorePath() {
+    return storePath;
+  }
+
+  public void setStorePath(String storePath) {
+    this.storePath = storePath;
+  }
+
+  /**
+   * to generate the hash code
+   */
+  @Override public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((databaseName == null) ? 0 : databaseName.hashCode());
+    result = prime * result + ((tableUniqueName == null) ? 0 : tableUniqueName.hashCode());
+    return result;
+  }
+
+  /**
+   * Overridden equals method
+   */
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof TableInfo)) {
+      return false;
+    }
+    TableInfo other = (TableInfo) obj;
+    if (databaseName == null) {
+      if (other.databaseName != null) {
+        return false;
+      }
+    } else if (!tableUniqueName.equals(other.tableUniqueName)) {
+      return false;
+    }
+
+    if (tableUniqueName == null) {
+      if (other.tableUniqueName != null) {
+        return false;
+      }
+    } else if (!tableUniqueName.equals(other.tableUniqueName)) {
+      return false;
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/TableSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/TableSchema.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/TableSchema.java
new file mode 100644
index 0000000..ded5635
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/TableSchema.java
@@ -0,0 +1,185 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.metadata.schema.table;
+
+import java.io.Serializable;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.metadata.schema.SchemaEvolution;
+import org.apache.carbondata.core.carbon.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.constants.CarbonCommonConstants;
+
+/**
+ * Persisting the table information
+ */
+public class TableSchema implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = -1928614587722507026L;
+
+  /**
+   * table id
+   */
+  private String tableId;
+
+  /**
+   * table Name
+   */
+  private String tableName;
+
+  /**
+   * Columns in the table
+   */
+  private List<ColumnSchema> listOfColumns;
+
+  /**
+   * History of schema evolution of this table
+   */
+  private SchemaEvolution schemaEvalution;
+
+  public TableSchema() {
+    this.listOfColumns = new ArrayList<ColumnSchema>(CarbonCommonConstants.DEFAULT_COLLECTION_SIZE);
+  }
+
+  /**
+   * @return the tableId
+   */
+  public String getTableId() {
+    return tableId;
+  }
+
+  /**
+   * @param tableId the tableId to set
+   */
+  public void setTableId(String tableId) {
+    this.tableId = tableId;
+  }
+
+  /**
+   * @return the listOfColumns
+   */
+  public List<ColumnSchema> getListOfColumns() {
+    return listOfColumns;
+  }
+
+  /**
+   * @param listOfColumns the listOfColumns to set
+   */
+  public void setListOfColumns(List<ColumnSchema> listOfColumns) {
+    this.listOfColumns = listOfColumns;
+  }
+
+  /**
+   * @return the schemaEvalution
+   */
+  public SchemaEvolution getSchemaEvalution() {
+    return schemaEvalution;
+  }
+
+  /**
+   * @param schemaEvalution the schemaEvalution to set
+   */
+  public void setSchemaEvalution(SchemaEvolution schemaEvalution) {
+    this.schemaEvalution = schemaEvalution;
+  }
+
+  /**
+   * @return the tableName
+   */
+  public String getTableName() {
+    return tableName;
+  }
+
+  /**
+   * @param tableName the tableName to set
+   */
+  public void setTableName(String tableName) {
+    this.tableName = tableName;
+  }
+
+  /**
+   * to get the column schema present in the table by name
+   *
+   * @param columnName
+   * @return column schema if matches the name
+   */
+  public ColumnSchema getColumnSchemaByName(String columnName) {
+    for (ColumnSchema tableColumn : listOfColumns) {
+      if (tableColumn.getColumnName().equals(columnName)) {
+        return tableColumn;
+      }
+    }
+    return null;
+  }
+
+  /**
+   * to get the column schema present in the table by unique id
+   *
+   * @param columnUniqueId
+   * @return column schema if matches the id
+   */
+  public ColumnSchema getColumnSchemaById(String columnUniqueId) {
+    for (ColumnSchema tableColumn : listOfColumns) {
+      if (tableColumn.getColumnUniqueId().equalsIgnoreCase(columnUniqueId)) {
+        return tableColumn;
+      }
+    }
+    return null;
+  }
+
+  @Override public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((tableId == null) ? 0 : tableId.hashCode());
+    result = prime * result + ((tableName == null) ? 0 : tableName.hashCode());
+    return result;
+  }
+
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (getClass() != obj.getClass()) {
+      return false;
+    }
+    TableSchema other = (TableSchema) obj;
+    if (tableId == null) {
+      if (other.tableId != null) {
+        return false;
+      }
+    } else if (!tableId.equals(other.tableId)) {
+      return false;
+    }
+    if (tableName == null) {
+      if (other.tableName != null) {
+        return false;
+      }
+    } else if (!tableName.equals(other.tableName)) {
+      return false;
+    }
+    return true;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java
new file mode 100644
index 0000000..88119d0
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/CarbonColumn.java
@@ -0,0 +1,174 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.carbondata.core.carbon.metadata.schema.table.column;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.carbon.ColumnIdentifier;
+import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
+
+public class CarbonColumn implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = 3648269871256322681L;
+
+  /**
+   * column schema
+   */
+  protected ColumnSchema columnSchema;
+
+  /**
+   * table ordinal
+   */
+  protected int ordinal;
+
+  /**
+   * default value for in case of restructuring will be used when older
+   * segment does not have particular column
+   */
+  protected byte[] defaultValue;
+
+  /**
+   * Column identifier
+   */
+  protected ColumnIdentifier columnIdentifier;
+
+  public CarbonColumn(ColumnSchema columnSchema, int ordinal) {
+    this.columnSchema = columnSchema;
+    this.ordinal = ordinal;
+    this.columnIdentifier =
+        new ColumnIdentifier(getColumnId(), getColumnProperties(), getDataType());
+  }
+
+  /**
+   * @return columnar or row based
+   */
+  public boolean isColumnar() {
+    return columnSchema.isColumnar();
+  }
+
+  /**
+   * @return column unique id
+   */
+  public String getColumnId() {
+    return columnSchema.getColumnUniqueId();
+  }
+
+  /**
+   * @return the dataType
+   */
+  public DataType getDataType() {
+    return columnSchema.getDataType();
+  }
+
+  /**
+   * @return the colName
+   */
+  public String getColName() {
+    return columnSchema.getColumnName();
+  }
+
+  /**
+   * @return the ordinal
+   */
+  public int getOrdinal() {
+    return ordinal;
+  }
+
+  /**
+   * @return the list of encoder used in dimension
+   */
+  public List<Encoding> getEncoder() {
+    return columnSchema.getEncodingList();
+  }
+
+  /**
+   * @return row group id if it is row based
+   */
+  public int columnGroupId() {
+    return columnSchema.getColumnGroupId();
+  }
+
+  /**
+   * @return the defaultValue
+   */
+  public byte[] getDefaultValue() {
+    return defaultValue;
+  }
+
+  /**
+   * @param defaultValue the defaultValue to set
+   */
+  public void setDefaultValue(byte[] defaultValue) {
+    this.defaultValue = defaultValue;
+  }
+
+  /**
+   * @param encoding
+   * @return true if contains the passing encoding
+   */
+  public boolean hasEncoding(Encoding encoding) {
+    return columnSchema.hasEncoding(encoding);
+  }
+
+  /**
+   * @return if DataType is ARRAY or STRUCT, this method return true, else
+   * false.
+   */
+  public Boolean isComplex() {
+    return columnSchema.isComplex();
+  }
+
+  /**
+   * @return if column is dimension return true, else false.
+   */
+  public Boolean isDimesion() {
+    return columnSchema.isDimensionColumn();
+  }
+
+  /**
+   * @return if column use inverted index return true, else false.
+   */
+  public Boolean isUseInvertedIndnex() {
+    return columnSchema.isUseInvertedIndex();
+  }
+  public ColumnSchema getColumnSchema() {
+    return this.columnSchema;
+  }
+
+  /**
+   * @return columnproperty
+   */
+  public Map<String, String> getColumnProperties() {
+    return this.columnSchema.getColumnProperties();
+  }
+
+  /**
+   * @return columnIdentifier
+   */
+  public ColumnIdentifier getColumnIdentifier() {
+    return this.columnIdentifier;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/CarbonDimension.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/CarbonDimension.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/CarbonDimension.java
new file mode 100644
index 0000000..c81cea4
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/CarbonDimension.java
@@ -0,0 +1,154 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.carbondata.core.carbon.metadata.schema.table.column;
+
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
+
+public class CarbonDimension extends CarbonColumn {
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = 3648269871656322681L;
+
+  /**
+   * List of child dimension for complex type
+   */
+  private List<CarbonDimension> listOfChildDimensions;
+
+  /**
+   * in case of dictionary dimension this will store the ordinal
+   * of the dimension in mdkey
+   */
+  private int keyOrdinal;
+
+  /**
+   * column group column ordinal
+   * for example if column is second column in the group
+   * it will store 2
+   */
+  private int columnGroupOrdinal;
+
+  /**
+   * to store complex type dimension ordinal
+   */
+  private int complexTypeOrdinal;
+
+  public CarbonDimension(ColumnSchema columnSchema, int ordinal, int keyOrdinal,
+      int columnGroupOrdinal, int complexTypeOrdinal) {
+    super(columnSchema, ordinal);
+    this.keyOrdinal = keyOrdinal;
+    this.columnGroupOrdinal = columnGroupOrdinal;
+    this.complexTypeOrdinal = complexTypeOrdinal;
+  }
+
+  /**
+   * this method will initialize list based on number of child dimensions Count
+   */
+  public void initializeChildDimensionsList(int childDimension) {
+    listOfChildDimensions = new ArrayList<CarbonDimension>(childDimension);
+  }
+
+  /**
+   * @return number of children for complex type
+   */
+  public int getNumberOfChild() {
+    return columnSchema.getNumberOfChild();
+  }
+
+  /**
+   * @return list of children dims for complex type
+   */
+  public List<CarbonDimension> getListOfChildDimensions() {
+    return listOfChildDimensions;
+  }
+
+  /**
+   * @return return the number of child present in case of complex type
+   */
+  public int numberOfChild() {
+    return columnSchema.getNumberOfChild();
+  }
+
+  public boolean hasEncoding(Encoding encoding) {
+    return columnSchema.getEncodingList().contains(encoding);
+  }
+
+  /**
+   * @return the keyOrdinal
+   */
+  public int getKeyOrdinal() {
+    return keyOrdinal;
+  }
+
+  /**
+   * @return the columnGroupOrdinal
+   */
+  public int getColumnGroupOrdinal() {
+    return columnGroupOrdinal;
+  }
+
+  /**
+   * @return the complexTypeOrdinal
+   */
+  public int getComplexTypeOrdinal() {
+    return complexTypeOrdinal;
+  }
+
+  public void setComplexTypeOridnal(int complexTypeOrdinal) {
+    this.complexTypeOrdinal = complexTypeOrdinal;
+  }
+
+  /**
+   * to generate the hash code for this class
+   */
+  @Override public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((columnSchema == null) ? 0 : columnSchema.hashCode());
+    return result;
+  }
+
+  /**
+   * to check whether to dimension are equal or not
+   */
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof CarbonDimension)) {
+      return false;
+    }
+    CarbonDimension other = (CarbonDimension) obj;
+    if (columnSchema == null) {
+      if (other.columnSchema != null) {
+        return false;
+      }
+    } else if (!columnSchema.equals(other.columnSchema)) {
+      return false;
+    }
+    return true;
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/CarbonMeasure.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/CarbonMeasure.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/CarbonMeasure.java
new file mode 100644
index 0000000..2c8c11e
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/CarbonMeasure.java
@@ -0,0 +1,112 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+
+package org.apache.carbondata.core.carbon.metadata.schema.table.column;
+
+/**
+ * class represent column(measure) in table
+ */
+public class CarbonMeasure extends CarbonColumn {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = 354341488059013977L;
+
+  /**
+   * aggregator chosen for measure
+   */
+  private String aggregateFunction;
+
+  /**
+   * Used when this column contains decimal data.
+   */
+  private int scale;
+
+  /**
+   * precision in decimal data
+   */
+  private int precision;
+
+  public CarbonMeasure(ColumnSchema columnSchema, int ordinal) {
+    super(columnSchema, ordinal);
+    this.scale = columnSchema.getScale();
+    this.precision = columnSchema.getPrecision();
+  }
+
+  /**
+   * @return the scale
+   */
+  public int getScale() {
+    return scale;
+  }
+
+  /**
+   * @return the precision
+   */
+  public int getPrecision() {
+    return precision;
+  }
+
+  /**
+   * @return the aggregator
+   */
+  public String getAggregateFunction() {
+    return aggregateFunction;
+  }
+
+  /**
+   * @param aggregateFunction the aggregateFunction to set
+   */
+  public void setAggregateFunction(String aggregateFunction) {
+    this.aggregateFunction = aggregateFunction;
+  }
+
+  /**
+   * to check whether to dimension are equal or not
+   */
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof CarbonMeasure)) {
+      return false;
+    }
+    CarbonMeasure other = (CarbonMeasure) obj;
+    if (columnSchema == null) {
+      if (other.columnSchema != null) {
+        return false;
+      }
+    } else if (!columnSchema.equals(other.columnSchema)) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * hash code
+   * @return
+   */
+  @Override public int hashCode() {
+    return this.getColumnSchema().getColumnUniqueId().hashCode();
+  }
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java
new file mode 100644
index 0000000..270702a
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/metadata/schema/table/column/ColumnSchema.java
@@ -0,0 +1,418 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.metadata.schema.table.column;
+
+import java.io.Serializable;
+import java.util.List;
+import java.util.Map;
+
+import org.apache.carbondata.core.carbon.metadata.datatype.DataType;
+import org.apache.carbondata.core.carbon.metadata.encoder.Encoding;
+
+/**
+ * Store the information about the column meta data present the table
+ */
+public class ColumnSchema implements Serializable {
+
+  /**
+   * serialization version
+   */
+  private static final long serialVersionUID = 7676766554874863763L;
+
+  /**
+   * dataType
+   */
+  private DataType dataType;
+  /**
+   * Name of the column. If it is a complex data type, we follow a naming rule
+   * grand_parent_column.parent_column.child_column
+   * For Array types, two columns will be stored one for
+   * the array type and one for the primitive type with
+   * the name parent_column.value
+   */
+  private String columnName;
+
+  /**
+   * Unique ID for a column. if this is dimension,
+   * it is an unique ID that used in dictionary
+   */
+  private String columnUniqueId;
+
+  /**
+   * column reference id
+   */
+  private String columnReferenceId;
+
+  /**
+   * whether it is stored as columnar format or row format
+   */
+  private boolean isColumnar = true;
+
+  /**
+   * List of encoding that are chained to encode the data for this column
+   */
+  private List<Encoding> encodingList;
+
+  /**
+   * Whether the column is a dimension or measure
+   */
+  private boolean isDimensionColumn;
+
+  /**
+   * Whether the column should use inverted index
+   */
+  private boolean useInvertedIndex;
+
+  /**
+   * The group ID for column used for row format columns,
+   * where in columns in each group are chunked together.
+   */
+  private int columnGroupId = -1;
+
+  /**
+   * Used when this column contains decimal data.
+   */
+  private int scale;
+
+  private int precision;
+
+  /**
+   * Nested fields.  Since thrift does not support nested fields,
+   * the nesting is flattened to a single list by a depth-first traversal.
+   * The children count is used to construct the nested relationship.
+   * This field is not set when the element is a primitive type
+   */
+  private int numberOfChild;
+
+  /**
+   * Used when this column is part of an aggregate function.
+   */
+  private String aggregateFunction;
+
+  /**
+   * used in case of schema restructuring
+   */
+  private byte[] defaultValue;
+
+  /**
+   * Column properties
+   */
+  private Map<String, String> columnProperties;
+
+  /**
+   * used to define the column visibility of column default is false
+   */
+  private boolean invisible = false;
+
+  /**
+   * @return the columnName
+   */
+  public String getColumnName() {
+    return columnName;
+  }
+
+  /**
+   * @param columnName the columnName to set
+   */
+  public void setColumnName(String columnName) {
+    this.columnName = columnName;
+  }
+
+  /**
+   * @return the columnUniqueId
+   */
+  public String getColumnUniqueId() {
+    return columnUniqueId;
+  }
+
+  /**
+   * @param columnUniqueId the columnUniqueId to set
+   */
+  public void setColumnUniqueId(String columnUniqueId) {
+    this.columnUniqueId = columnUniqueId;
+  }
+
+  /**
+   * @return the isColumnar
+   */
+  public boolean isColumnar() {
+    return isColumnar;
+  }
+
+  /**
+   * @param isColumnar the isColumnar to set
+   */
+  public void setColumnar(boolean isColumnar) {
+    this.isColumnar = isColumnar;
+  }
+
+  /**
+   * @return the isDimensionColumn
+   */
+  public boolean isDimensionColumn() {
+    return isDimensionColumn;
+  }
+
+  /**
+   * @param isDimensionColumn the isDimensionColumn to set
+   */
+  public void setDimensionColumn(boolean isDimensionColumn) {
+    this.isDimensionColumn = isDimensionColumn;
+  }
+
+  /**
+   * the isUseInvertedIndex
+   */
+  public boolean isUseInvertedIndex() {
+    return useInvertedIndex;
+  }
+
+  /**
+   * @param useInvertedIndex the useInvertedIndex to set
+   */
+  public void setUseInvertedIndex(boolean useInvertedIndex) {
+    this.useInvertedIndex = useInvertedIndex;
+  }
+
+  /**
+   * @return the columnGroup
+   */
+  public int getColumnGroupId() {
+    return columnGroupId;
+  }
+
+  /**
+   * @param columnGroup the columnGroup to set
+   */
+  public void setColumnGroup(int columnGroupId) {
+    this.columnGroupId = columnGroupId;
+  }
+
+  /**
+   * @return the scale
+   */
+  public int getScale() {
+    return scale;
+  }
+
+  /**
+   * @param scale the scale to set
+   */
+  public void setScale(int scale) {
+    this.scale = scale;
+  }
+
+  /**
+   * @return the precision
+   */
+  public int getPrecision() {
+    return precision;
+  }
+
+  /**
+   * @param precision the precision to set
+   */
+  public void setPrecision(int precision) {
+    this.precision = precision;
+  }
+
+  /**
+   * @return the numberOfChild
+   */
+  public int getNumberOfChild() {
+    return numberOfChild;
+  }
+
+  /**
+   * @param numberOfChild the numberOfChild to set
+   */
+  public void setNumberOfChild(int numberOfChild) {
+    this.numberOfChild = numberOfChild;
+  }
+
+  /**
+   * @return the aggregator
+   */
+  public String getAggregateFunction() {
+    return aggregateFunction;
+  }
+
+  /**
+   * @param aggregateFunction the aggregator to set
+   */
+  public void setAggregateFunction(String aggregateFunction) {
+    this.aggregateFunction = aggregateFunction;
+  }
+
+  /**
+   * @return the defaultValue
+   */
+  public byte[] getDefaultValue() {
+    return defaultValue;
+  }
+
+  /**
+   * @param defaultValue the defaultValue to set
+   */
+  public void setDefaultValue(byte[] defaultValue) {
+    this.defaultValue = defaultValue;
+  }
+
+  /**
+   * hash code method to check get the hashcode based.
+   * for generating the hash code only column name and column unique id will considered
+   */
+  @Override public int hashCode() {
+    final int prime = 31;
+    int result = 1;
+    result = prime * result + ((columnName == null) ? 0 : columnName.hashCode());
+    return result;
+  }
+
+  /**
+   * Overridden equals method for columnSchema
+   */
+  @Override public boolean equals(Object obj) {
+    if (this == obj) {
+      return true;
+    }
+    if (obj == null) {
+      return false;
+    }
+    if (!(obj instanceof ColumnSchema)) {
+      return false;
+    }
+    ColumnSchema other = (ColumnSchema) obj;
+    if (columnName == null) {
+      if (other.columnName != null) {
+        return false;
+      }
+    } else if (!columnName.equals(other.columnName)) {
+      return false;
+    }
+    return true;
+  }
+
+  /**
+   * @return the dataType
+   */
+  public DataType getDataType() {
+    return dataType;
+  }
+
+  /**
+   * @param dataType the dataType to set
+   */
+  public void setDataType(DataType dataType) {
+    this.dataType = dataType;
+  }
+
+  /**
+   * @return the encoderList
+   */
+  public List<Encoding> getEncodingList() {
+    return encodingList;
+  }
+
+  /**
+   * @param encoderList the encoderList to set
+   */
+  public void setEncodingList(List<Encoding> encodingList) {
+    this.encodingList = encodingList;
+  }
+
+  /**
+   * @param encoding
+   * @return true if contains the passing encoding
+   */
+  public boolean hasEncoding(Encoding encoding) {
+    if (encodingList == null || encodingList.isEmpty()) {
+      return false;
+    } else {
+      return encodingList.contains(encoding);
+    }
+  }
+
+  /**
+   * @return if DataType is ARRAY or STRUCT, this method return true, else
+   * false.
+   */
+  public Boolean isComplex() {
+    if (DataType.ARRAY.equals(this.getDataType()) || DataType.STRUCT.equals(this.getDataType())) {
+      return true;
+    } else {
+      return false;
+    }
+  }
+
+  /**
+   * @param columnProperties
+   */
+  public void setColumnProperties(Map<String, String> columnProperties) {
+    this.columnProperties = columnProperties;
+  }
+
+  /**
+   * @param property
+   * @return
+   */
+  public String getColumnProperty(String property) {
+    if (null != columnProperties) {
+      return columnProperties.get(property);
+    }
+    return null;
+  }
+
+  /**
+   * return columnproperties
+   */
+  public Map<String, String> getColumnProperties() {
+    return columnProperties;
+  }
+  /**
+   * return the visibility
+   * @return
+   */
+  public boolean isInvisible() {
+    return invisible;
+  }
+
+  /**
+   * set the visibility
+   * @param invisible
+   */
+  public void setInvisible(boolean invisible) {
+    this.invisible = invisible;
+  }
+
+  /**
+   * @return columnReferenceId
+   */
+  public String getColumnReferenceId() {
+    return columnReferenceId;
+  }
+
+  /**
+   * @param columnReferenceId
+   */
+  public void setColumnReferenceId(String columnReferenceId) {
+    this.columnReferenceId = columnReferenceId;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/path/CarbonSharedDictionaryPath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/path/CarbonSharedDictionaryPath.java b/core/src/main/java/org/apache/carbondata/core/carbon/path/CarbonSharedDictionaryPath.java
new file mode 100644
index 0000000..7be92bc
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/path/CarbonSharedDictionaryPath.java
@@ -0,0 +1,73 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.path;
+
+import java.io.File;
+
+/**
+ * Helps to get Shared dimension files path.
+ */
+public class CarbonSharedDictionaryPath {
+
+  private static final String SHAREDDIM_DIR = "SharedDictionary";
+  private static final String DICTIONARY_EXT = ".dict";
+  private static final String DICTIONARY_META_EXT = ".dictmeta";
+  private static final String SORT_INDEX_EXT = ".sortindex";
+
+  /***
+   * @param storePath    store path
+   * @param databaseName data base name
+   * @param columnId     unique column identifier
+   * @return absolute path of shared dictionary file
+   */
+  public static String getDictionaryFilePath(String storePath, String databaseName,
+      String columnId) {
+    return getSharedDictionaryDir(storePath, databaseName) + File.separator + columnId
+        + DICTIONARY_EXT;
+  }
+
+  /***
+   * @param storePath    store path
+   * @param databaseName data base name
+   * @param columnId     unique column identifier
+   * @return absolute path of shared dictionary meta file
+   */
+  public static String getDictionaryMetaFilePath(String storePath, String databaseName,
+      String columnId) {
+    return getSharedDictionaryDir(storePath, databaseName) + File.separator + columnId
+        + DICTIONARY_META_EXT;
+  }
+
+  /***
+   * @param storePath    store path
+   * @param databaseName data base name
+   * @param columnId     unique column identifier
+   * @return absolute path of shared dictionary sort index file
+   */
+  public static String getSortIndexFilePath(String storePath, String databaseName,
+      String columnId) {
+    return getSharedDictionaryDir(storePath, databaseName) + File.separator + columnId
+        + SORT_INDEX_EXT;
+  }
+
+  private static String getSharedDictionaryDir(String storePath, String databaseName) {
+    return storePath + File.separator + databaseName + File.separator + SHAREDDIM_DIR;
+  }
+
+}

http://git-wip-us.apache.org/repos/asf/incubator-carbondata/blob/cd6a4ff3/core/src/main/java/org/apache/carbondata/core/carbon/path/CarbonStorePath.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/carbon/path/CarbonStorePath.java b/core/src/main/java/org/apache/carbondata/core/carbon/path/CarbonStorePath.java
new file mode 100644
index 0000000..567602b
--- /dev/null
+++ b/core/src/main/java/org/apache/carbondata/core/carbon/path/CarbonStorePath.java
@@ -0,0 +1,69 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *    http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.carbondata.core.carbon.path;
+
+import java.io.File;
+
+import org.apache.carbondata.core.carbon.CarbonTableIdentifier;
+
+import org.apache.hadoop.fs.Path;
+
+/**
+ * Helps to get Store content paths.
+ */
+public class CarbonStorePath extends Path {
+
+  private String storePath;
+
+  public CarbonStorePath(String storePathString) {
+    super(storePathString);
+    this.storePath = storePathString;
+  }
+
+  /**
+   * gets CarbonTablePath object to manage table paths
+   */
+  public static CarbonTablePath getCarbonTablePath(String storePath,
+      CarbonTableIdentifier tableIdentifier) {
+    CarbonTablePath carbonTablePath = new CarbonTablePath(tableIdentifier,
+        storePath + File.separator + tableIdentifier.getDatabaseName() + File.separator
+            + tableIdentifier.getTableName());
+
+    return carbonTablePath;
+  }
+
+  /**
+   * gets CarbonTablePath object to manage table paths
+   */
+  public CarbonTablePath getCarbonTablePath(CarbonTableIdentifier tableIdentifier) {
+    return CarbonStorePath.getCarbonTablePath(storePath, tableIdentifier);
+  }
+
+  @Override public boolean equals(Object o) {
+    if (!(o instanceof CarbonStorePath)) {
+      return false;
+    }
+    CarbonStorePath path = (CarbonStorePath)o;
+    return storePath.equals(path.storePath) && super.equals(o);
+  }
+
+  @Override public int hashCode() {
+    return super.hashCode() + storePath.hashCode();
+  }
+}