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 2018/02/07 11:49:17 UTC

[2/4] carbondata-site git commit: update documents for 1.3.0

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/711502d1/src/site/markdown/data-management-on-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/data-management-on-carbondata.md b/src/site/markdown/data-management-on-carbondata.md
new file mode 100644
index 0000000..18ad5b8
--- /dev/null
+++ b/src/site/markdown/data-management-on-carbondata.md
@@ -0,0 +1,1219 @@
+<!--
+    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.
+-->
+
+# Data Management on CarbonData
+
+This tutorial is going to introduce all commands and data operations on CarbonData.
+
+* [CREATE TABLE](#create-table)
+* [CREATE DATABASE](#create-database)
+* [TABLE MANAGEMENT](#table-management)
+* [LOAD DATA](#load-data)
+* [UPDATE AND DELETE](#update-and-delete)
+* [COMPACTION](#compaction)
+* [PARTITION](#partition)
+* [PRE-AGGREGATE TABLES](#pre-aggregate-tables)
+* [BUCKETING](#bucketing)
+* [SEGMENT MANAGEMENT](#segment-management)
+
+## CREATE TABLE
+
+  This command can be used to create a CarbonData table by specifying the list of fields along with the table properties. You can also specify the location where the table needs to be stored.
+  
+  ```
+  CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type , ...)]
+  STORED BY 'carbondata'
+  [TBLPROPERTIES (property_name=property_value, ...)]
+  [LOCATION 'path']
+  ```  
+  
+### Usage Guidelines
+
+  Following are the guidelines for TBLPROPERTIES, CarbonData's additional table options can be set via carbon.properties.
+  
+   - **Dictionary Encoding Configuration**
+
+     Dictionary encoding is turned off for all columns by default from 1.3 onwards, you can use this command for including or excluding columns to do dictionary encoding.
+     Suggested use cases : do dictionary encoding for low cardinality columns, it might help to improve data compression ratio and performance.
+
+     ```
+     TBLPROPERTIES ('DICTIONARY_INCLUDE'='column1, column2')
+	 ```
+	 
+   - **Inverted Index Configuration**
+
+     By default inverted index is enabled, it might help to improve compression ratio and query speed, especially for low cardinality columns which are in reward position.
+     Suggested use cases : For high cardinality columns, you can disable the inverted index for improving the data loading performance.
+
+     ```
+     TBLPROPERTIES ('NO_INVERTED_INDEX'='column1, column3')
+     ```
+
+   - **Sort Columns Configuration**
+
+     This property is for users to specify which columns belong to the MDK(Multi-Dimensions-Key) index.
+     * If users don't specify "SORT_COLUMN" property, by default MDK index be built by using all dimension columns except complex data type column. 
+     * If this property is specified but with empty argument, then the table will be loaded without sort.
+	 * This supports only string, date, timestamp, short, int, long, and boolean data types.
+     Suggested use cases : Only build MDK index for required columns,it might help to improve the data loading performance.
+
+     ```
+     TBLPROPERTIES ('SORT_COLUMNS'='column1, column3')
+     OR
+     TBLPROPERTIES ('SORT_COLUMNS'='')
+     ```
+
+   - **Sort Scope Configuration**
+   
+     This property is for users to specify the scope of the sort during data load, following are the types of sort scope.
+     
+     * LOCAL_SORT: It is the default sort scope.             
+     * NO_SORT: It will load the data in unsorted manner, it will significantly increase load performance.       
+     * BATCH_SORT: It increases the load performance but decreases the query performance if identified blocks > parallelism.
+     * GLOBAL_SORT: It increases the query performance, especially high concurrent point query.
+       And if you care about loading resources isolation strictly, because the system uses the spark GroupBy to sort data, the resource can be controlled by spark. 
+ 
+   - **Table Block Size Configuration**
+
+     This command is for setting block size of this table, the default value is 1024 MB and supports a range of 1 MB to 2048 MB.
+
+     ```
+     TBLPROPERTIES ('TABLE_BLOCKSIZE'='512')
+     ```
+     NOTE: 512 or 512M both are accepted.
+
+   - **Table Compaction Configuration**
+   
+     These properties are table level compaction configurations, if not specified, system level configurations in carbon.properties will be used.
+     Following are 5 configurations:
+     
+     * MAJOR_COMPACTION_SIZE: same meaning with carbon.major.compaction.size, size in MB.
+     * AUTO_LOAD_MERGE: same meaning with carbon.enable.auto.load.merge.
+     * COMPACTION_LEVEL_THRESHOLD: same meaning with carbon.compaction.level.threshold.
+     * COMPACTION_PRESERVE_SEGMENTS: same meaning with carbon.numberof.preserve.segments.
+     * ALLOWED_COMPACTION_DAYS: same meaning with carbon.allowed.compaction.days.     
+
+     ```
+     TBLPROPERTIES ('MAJOR_COMPACTION_SIZE'='2048',
+                    'AUTO_LOAD_MERGE'='true',
+                    'COMPACTION_LEVEL_THRESHOLD'='5,6',
+                    'COMPACTION_PRESERVE_SEGMENTS'='10',
+                    'ALLOWED_COMPACTION_DAYS'='5')
+     ```
+     
+   - **Streaming**
+
+     CarbonData supports streaming ingestion for real-time data. You can create the ‘streaming’ table using the following table properties.
+
+     ```
+     TBLPROPERTIES ('streaming'='true')
+     ```
+
+### Example:
+
+   ```
+    CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
+                                   productNumber Int,
+                                   productName String,
+                                   storeCity String,
+                                   storeProvince String,
+                                   productCategory String,
+                                   productBatch String,
+                                   saleQuantity Int,
+                                   revenue Int)
+    STORED BY 'carbondata'
+    TBLPROPERTIES ('DICTIONARY_INCLUDE'='productNumber',
+                   'NO_INVERTED_INDEX'='productBatch',
+                   'SORT_COLUMNS'='productName,storeCity',
+                   'SORT_SCOPE'='NO_SORT',
+                   'TABLE_BLOCKSIZE'='512',
+                   'MAJOR_COMPACTION_SIZE'='2048',
+                   'AUTO_LOAD_MERGE'='true',
+                   'COMPACTION_LEVEL_THRESHOLD'='5,6',
+                   'COMPACTION_PRESERVE_SEGMENTS'='10',
+				   'streaming'='true',
+                   'ALLOWED_COMPACTION_DAYS'='5')
+   ```
+
+## CREATE DATABASE 
+  This function creates a new database. By default the database is created in Carbon store location, but you can also specify custom location.
+  ```
+  CREATE DATABASE [IF NOT EXISTS] database_name [LOCATION path];
+  ```
+  
+### Example
+  ```
+  CREATE DATABASE carbon LOCATION “hdfs://name_cluster/dir1/carbonstore”;
+  ```
+
+## CREATE TABLE As SELECT
+  This function allows you to create a Carbon table from any of the Parquet/Hive/Carbon table. This is beneficial when the user wants to create Carbon table from any other Parquet/Hive table and use the Carbon query engine to query and achieve better query results for cases where Carbon is faster than other file formats. Also this feature can be used for backing up the data.
+  ```
+  CREATE TABLE [IF NOT EXISTS] [db_name.]table_name STORED BY 'carbondata' [TBLPROPERTIES (key1=val1, key2=val2, ...)] AS select_statement;
+  ```
+
+### Examples
+  ```
+  CREATE TABLE ctas_select_parquet STORED BY 'carbondata' as select * from parquet_ctas_test;
+  ```
+   
+## TABLE MANAGEMENT  
+
+### SHOW TABLE
+
+  This command can be used to list all the tables in current database or all the tables of a specific database.
+  ```
+  SHOW TABLES [IN db_Name]
+  ```
+
+  Example:
+  ```
+  SHOW TABLES
+  OR
+  SHOW TABLES IN defaultdb
+  ```
+
+### ALTER TALBE
+
+  The following section introduce the commands to modify the physical or logical state of the existing table(s).
+
+   - **RENAME TABLE**
+   
+     This command is used to rename the existing table.
+     ```
+     ALTER TABLE [db_name.]table_name RENAME TO new_table_name
+     ```
+
+     Examples:
+     ```
+     ALTER TABLE carbon RENAME TO carbondata
+     OR
+     ALTER TABLE test_db.carbon RENAME TO test_db.carbondata
+     ```
+
+   - **ADD COLUMNS**
+   
+     This command is used to add a new column to the existing table.
+     ```
+     ALTER TABLE [db_name.]table_name ADD COLUMNS (col_name data_type,...)
+     TBLPROPERTIES('DICTIONARY_INCLUDE'='col_name,...',
+     'DEFAULT.VALUE.COLUMN_NAME'='default_value')
+     ```
+
+     Examples:
+     ```
+     ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING)
+     ```
+
+     ```
+     ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DICTIONARY_INCLUDE'='a1')
+     ```
+
+     ```
+     ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DEFAULT.VALUE.a1'='10')
+     ```
+
+   - **DROP COLUMNS**
+   
+     This command is used to delete the existing column(s) in a table.
+     ```
+     ALTER TABLE [db_name.]table_name DROP COLUMNS (col_name, ...)
+     ```
+
+     Examples:
+     ```
+     ALTER TABLE carbon DROP COLUMNS (b1)
+     OR
+     ALTER TABLE test_db.carbon DROP COLUMNS (b1)
+     
+     ALTER TABLE carbon DROP COLUMNS (c1,d1)
+     ```
+
+   - **CHANGE DATA TYPE**
+   
+     This command is used to change the data type from INT to BIGINT or decimal precision from lower to higher.
+     Change of decimal data type from lower precision to higher precision will only be supported for cases where there is no data loss.
+     ```
+     ALTER TABLE [db_name.]table_name CHANGE col_name col_name changed_column_type
+     ```
+
+     Valid Scenarios
+     - Invalid scenario - Change of decimal precision from (10,2) to (10,5) is invalid as in this case only scale is increased but total number of digits remains the same.
+     - Valid scenario - Change of decimal precision from (10,2) to (12,3) is valid as the total number of digits are increased by 2 but scale is increased only by 1 which will not lead to any data loss.
+     - NOTE: The allowed range is 38,38 (precision, scale) and is a valid upper case scenario which is not resulting in data loss.
+
+     Example1:Changing data type of column a1 from INT to BIGINT.
+     ```
+     ALTER TABLE test_db.carbon CHANGE a1 a1 BIGINT
+     ```
+     
+     Example2:Changing decimal precision of column a1 from 10 to 18.
+     ```
+     ALTER TABLE test_db.carbon CHANGE a1 a1 DECIMAL(18,2)
+     ```
+
+### DROP TABLE
+  
+  This command is used to delete an existing table.
+  ```
+  DROP TABLE [IF EXISTS] [db_name.]table_name
+  ```
+
+  Example:
+  ```
+  DROP TABLE IF EXISTS productSchema.productSalesTable
+  ```
+ 
+### REFRESH TABLE
+ 
+  This command is used to register Carbon table to HIVE meta store catalogue from existing Carbon table data.
+  ```
+  REFRESH TABLE $db_NAME.$table_NAME
+  ```
+  
+  Example:
+  ```
+  REFRESH TABLE dbcarbon.productSalesTable
+  ```
+  NOTE: 
+  * The new database name and the old database name should be same.
+  * Before executing this command the old table schema and data should be copied into the new database location.
+  * If the table is aggregate table, then all the aggregate tables should be copied to the new database location.
+  * For old store, the time zone of the source and destination cluster should be same.
+  * If old cluster uses HIVE meta store, refresh will not work as schema file does not exist in file system.
+  
+
+## LOAD DATA
+
+### LOAD FILES TO CARBONDATA TABLE
+  
+  This command is used to load csv files to carbondata, OPTIONS are not mandatory for data loading process. 
+  Inside OPTIONS user can provide either of any options like DELIMITER, QUOTECHAR, FILEHEADER, ESCAPECHAR, MULTILINE as per requirement.
+  
+  ```
+  LOAD DATA [LOCAL] INPATH 'folder_path' 
+  INTO TABLE [db_name.]table_name 
+  OPTIONS(property_name=property_value, ...)
+  ```
+
+  You can use the following options to load data:
+  
+  - **DELIMITER:** Delimiters can be provided in the load command.
+
+    ``` 
+    OPTIONS('DELIMITER'=',')
+    ```
+
+  - **QUOTECHAR:** Quote Characters can be provided in the load command.
+
+    ```
+    OPTIONS('QUOTECHAR'='"')
+    ```
+
+  - **COMMENTCHAR:** Comment Characters can be provided in the load command if user want to comment lines.
+
+    ```
+    OPTIONS('COMMENTCHAR'='#')
+    ```
+
+  - **FILEHEADER:** Headers can be provided in the LOAD DATA command if headers are missing in the source files.
+
+    ```
+    OPTIONS('FILEHEADER'='column1,column2') 
+    ```
+
+  - **MULTILINE:** CSV with new line character in quotes.
+
+    ```
+    OPTIONS('MULTILINE'='true') 
+    ```
+
+  - **ESCAPECHAR:** Escape char can be provided if user want strict validation of escape character on CSV.
+
+    ```
+    OPTIONS('ESCAPECHAR'='\') 
+    ```
+  - **SKIP_EMPTY_LINE:** This option will ignore the empty line in the CSV file during the data load.
+
+    ```
+    OPTIONS('SKIP_EMPTY_LINE'='TRUE/FALSE') 
+    ```
+
+  - **COMPLEX_DELIMITER_LEVEL_1:** Split the complex type data column in a row (eg., a$b$c --> Array = {a,b,c}).
+
+    ```
+    OPTIONS('COMPLEX_DELIMITER_LEVEL_1'='$') 
+    ```
+
+  - **COMPLEX_DELIMITER_LEVEL_2:** Split the complex type nested data column in a row. Applies level_1 delimiter & applies level_2 based on complex data type (eg., a:b$c:d --> Array> = {{a,b},{c,d}}).
+
+    ```
+    OPTIONS('COMPLEX_DELIMITER_LEVEL_2'=':')
+    ```
+
+  - **ALL_DICTIONARY_PATH:** All dictionary files path.
+
+    ```
+    OPTIONS('ALL_DICTIONARY_PATH'='/opt/alldictionary/data.dictionary')
+    ```
+
+  - **COLUMNDICT:** Dictionary file path for specified column.
+
+    ```
+    OPTIONS('COLUMNDICT'='column1:dictionaryFilePath1,column2:dictionaryFilePath2')
+    ```
+    NOTE: ALL_DICTIONARY_PATH and COLUMNDICT can't be used together.
+    
+  - **DATEFORMAT/TIMESTAMPFORMAT:** Date and Timestamp format for specified column.
+
+    ```
+    OPTIONS('DATEFORMAT' = 'yyyy-MM-dd','TIMESTAMPFORMAT'='yyyy-MM-dd HH:mm:ss')
+    ```
+    NOTE: Date formats are specified by date pattern strings. The date pattern letters in CarbonData are same as in JAVA. Refer to [SimpleDateFormat](http://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html).
+
+  - **SINGLE_PASS:** Single Pass Loading enables single job to finish data loading with dictionary generation on the fly. It enhances performance in the scenarios where the subsequent data loading after initial load involves fewer incremental updates on the dictionary.
+
+  This option specifies whether to use single pass for loading data or not. By default this option is set to FALSE.
+
+   ```
+    OPTIONS('SINGLE_PASS'='TRUE')
+   ```
+
+   NOTE:
+   * If this option is set to TRUE then data loading will take less time.
+   * If this option is set to some invalid value other than TRUE or FALSE then it uses the default value.
+
+   Example:
+
+   ```
+   LOAD DATA local inpath '/opt/rawdata/data.csv' INTO table carbontable
+   options('DELIMITER'=',', 'QUOTECHAR'='"','COMMENTCHAR'='#',
+   'FILEHEADER'='empno,empname,designation,doj,workgroupcategory,
+   workgroupcategoryname,deptno,deptname,projectcode,
+   projectjoindate,projectenddate,attendance,utilization,salary',
+   'MULTILINE'='true','ESCAPECHAR'='\','COMPLEX_DELIMITER_LEVEL_1'='$',
+   'COMPLEX_DELIMITER_LEVEL_2'=':',
+   'ALL_DICTIONARY_PATH'='/opt/alldictionary/data.dictionary',
+   'SINGLE_PASS'='TRUE')
+   ```
+
+  - **BAD RECORDS HANDLING:** Methods of handling bad records are as follows:
+
+    * Load all of the data before dealing with the errors.
+    * Clean or delete bad records before loading data or stop the loading when bad records are found.
+
+    ```
+    OPTIONS('BAD_RECORDS_LOGGER_ENABLE'='true', 'BAD_RECORD_PATH'='hdfs://hacluster/tmp/carbon', 'BAD_RECORDS_ACTION'='REDIRECT', 'IS_EMPTY_DATA_BAD_RECORD'='false')
+    ```
+
+  NOTE:
+  * BAD_RECORDS_ACTION property can have four type of actions for bad records FORCE, REDIRECT, IGNORE and FAIL.
+  * FAIL option is its Default value. If the FAIL option is used, then data loading fails if any bad records are found.
+  * If the REDIRECT option is used, CarbonData will add all bad records in to a separate CSV file. However, this file must not be used for subsequent data loading because the content may not exactly match the source record. You are advised to cleanse the original source record for further data ingestion. This option is used to remind you which records are bad records.
+  * If the FORCE option is used, then it auto-corrects the data by storing the bad records as NULL before Loading data.
+  * If the IGNORE option is used, then bad records are neither loaded nor written to the separate CSV file.
+  * In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails.
+  * The maximum number of characters per column is 100000. If there are more than 100000 characters in a column, data loading will fail.
+
+  Example:
+
+  ```
+  LOAD DATA INPATH 'filepath.csv' INTO TABLE tablename
+  OPTIONS('BAD_RECORDS_LOGGER_ENABLE'='true','BAD_RECORD_PATH'='hdfs://hacluster/tmp/carbon',
+  'BAD_RECORDS_ACTION'='REDIRECT','IS_EMPTY_DATA_BAD_RECORD'='false')
+  ```
+
+### INSERT DATA INTO CARBONDATA TABLE
+
+  This command inserts data into a CarbonData table, it is defined as a combination of two queries Insert and Select query respectively. 
+  It inserts records from a source table into a target CarbonData table, the source table can be a Hive table, Parquet table or a CarbonData table itself. 
+  It comes with the functionality to aggregate the records of a table by performing Select query on source table and load its corresponding resultant records into a CarbonData table.
+
+  ```
+  INSERT INTO TABLE <CARBONDATA TABLE> SELECT * FROM sourceTableName 
+  [ WHERE { <filter_condition> } ]
+  ```
+
+  You can also omit the `table` keyword and write your query as:
+ 
+  ```
+  INSERT INTO <CARBONDATA TABLE> SELECT * FROM sourceTableName 
+  [ WHERE { <filter_condition> } ]
+  ```
+
+  Overwrite insert data:
+  ```
+  INSERT OVERWRITE <CARBONDATA TABLE> SELECT * FROM sourceTableName 
+  [ WHERE { <filter_condition> } ]
+  ```
+
+  NOTE:
+  * The source table and the CarbonData table must have the same table schema.
+  * The data type of source and destination table columns should be same
+  * INSERT INTO command does not support partial success if bad records are found, it will fail.
+  * Data cannot be loaded or updated in source table while insert from source table to target table is in progress.
+
+  Examples
+  ```
+  INSERT INTO table1 SELECT item1, sum(item2 + 1000) as result FROM table2 group by item1
+  ```
+
+  ```
+  INSERT INTO table1 SELECT item1, item2, item3 FROM table2 where item2='xyz'
+  ```
+
+  ```
+  INSERT OVERWRITE table1 SELECT * FROM TABLE2
+  ```
+
+## UPDATE AND DELETE
+  
+### UPDATE
+  
+  This command will allow to update the CarbonData table based on the column expression and optional filter conditions.
+    
+  ```
+  UPDATE <table_name> 
+  SET (column_name1, column_name2, ... column_name n) = (column1_expression , column2_expression, ... column n_expression )
+  [ WHERE { <filter_condition> } ]
+  ```
+  
+  alternatively the following the command can also be used for updating the CarbonData Table :
+  
+  ```
+  UPDATE <table_name>
+  SET (column_name1, column_name2) =(select sourceColumn1, sourceColumn2 from sourceTable [ WHERE { <filter_condition> } ] )
+  [ WHERE { <filter_condition> } ]
+  ```
+  
+  NOTE:The update command fails if multiple input rows in source table are matched with single row in destination table.
+  
+  Examples:
+  ```
+  UPDATE t3 SET (t3_salary) = (t3_salary + 9) WHERE t3_name = 'aaa1'
+  ```
+  
+  ```
+  UPDATE t3 SET (t3_date, t3_country) = ('2017-11-18', 'india') WHERE t3_salary < 15003
+  ```
+  
+  ```
+  UPDATE t3 SET (t3_country, t3_name) = (SELECT t5_country, t5_name FROM t5 WHERE t5_id = 5) WHERE t3_id < 5
+  ```
+  
+  ```
+  UPDATE t3 SET (t3_date, t3_serialname, t3_salary) = (SELECT '2099-09-09', t5_serialname, '9999' FROM t5 WHERE t5_id = 5) WHERE t3_id < 5
+  ```
+  
+  
+  ```
+  UPDATE t3 SET (t3_country, t3_salary) = (SELECT t5_country, t5_salary FROM t5 FULL JOIN t3 u WHERE u.t3_id = t5_id and t5_id=6) WHERE t3_id >6
+  ```
+    
+### DELETE
+
+  This command allows us to delete records from CarbonData table.
+  ```
+  DELETE FROM table_name [WHERE expression]
+  ```
+  
+  Examples:
+  
+  ```
+  DELETE FROM carbontable WHERE column1  = 'china'
+  ```
+  
+  ```
+  DELETE FROM carbontable WHERE column1 IN ('china', 'USA')
+  ```
+  
+  ```
+  DELETE FROM carbontable WHERE column1 IN (SELECT column11 FROM sourceTable2)
+  ```
+  
+  ```
+  DELETE FROM carbontable WHERE column1 IN (SELECT column11 FROM sourceTable2 WHERE column1 = 'USA')
+  ```
+
+## COMPACTION
+
+  Compaction improves the query performance significantly. 
+  During the load data, several CarbonData files are generated, this is because data is sorted only within each load (per load segment and one B+ tree index).
+  
+  There are two types of compaction, Minor and Major compaction.
+  
+  ```
+  ALTER TABLE [db_name.]table_name COMPACT 'MINOR/MAJOR'
+  ```
+
+  - **Minor Compaction**
+  
+  In Minor compaction, user can specify the number of loads to be merged. 
+  Minor compaction triggers for every data load if the parameter carbon.enable.auto.load.merge is set to true. 
+  If any segments are available to be merged, then compaction will run parallel with data load, there are 2 levels in minor compaction:
+  * Level 1: Merging of the segments which are not yet compacted.
+  * Level 2: Merging of the compacted segments again to form a larger segment.
+  
+  ```
+  ALTER TABLE table_name COMPACT 'MINOR'
+  ```
+  
+  - **Major Compaction**
+  
+  In Major compaction, multiple segments can be merged into one large segment. 
+  User will specify the compaction size until which segments can be merged, Major compaction is usually done during the off-peak time.
+  This command merges the specified number of segments into one segment: 
+     
+  ```
+  ALTER TABLE table_name COMPACT 'MAJOR'
+  ```
+
+  - **CLEAN SEGMENTS AFTER Compaction**
+  
+  Clean the segments which are compacted:
+  ```
+  CLEAN FILES FOR TABLE carbon_table
+  ```
+
+## PARTITION
+
+### STANDARD PARTITION
+
+  The partition is similar as spark and hive partition, user can use any column to build partition:
+  
+#### Create Partition Table
+
+  This command allows you to create table with partition.
+  
+  ```
+  CREATE TABLE [IF NOT EXISTS] [db_name.]table_name 
+    [(col_name data_type , ...)]
+    [COMMENT table_comment]
+    [PARTITIONED BY (col_name data_type , ...)]
+    [STORED BY file_format]
+    [TBLPROPERTIES (property_name=property_value, ...)]
+  ```
+  
+  Example:
+  ```
+   CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
+                                productNumber Int,
+                                productName String,
+                                storeCity String,
+                                storeProvince String,
+                                saleQuantity Int,
+                                revenue Int)
+  PARTITIONED BY (productCategory String, productBatch String)
+  STORED BY 'carbondata'
+  ```
+		
+#### Load Data Using Static Partition 
+
+  This command allows you to load data using static partition.
+  
+  ```
+  LOAD DATA [LOCAL] INPATH 'folder_path' 
+    INTO TABLE [db_name.]table_name PARTITION (partition_spec) 
+    OPTIONS(property_name=property_value, ...)
+  NSERT INTO INTO TABLE [db_name.]table_name PARTITION (partition_spec) SELECT STATMENT 
+  ```
+  
+  Example:
+  ```
+  LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.txt'
+    INTO TABLE locationTable
+    PARTITION (country = 'US', state = 'CA')
+    
+  INSERT INTO TABLE locationTable
+    PARTITION (country = 'US', state = 'AL')
+    SELECT * FROM another_user au 
+    WHERE au.country = 'US' AND au.state = 'AL';
+  ```
+
+#### Load Data Using Dynamic Partition
+
+  This command allows you to load data using dynamic partition. If partition spec is not specified, then the partition is considered as dynamic.
+
+  Example:
+  ```
+  LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.txt'
+    INTO TABLE locationTable
+          
+  INSERT INTO TABLE locationTable
+    SELECT * FROM another_user au 
+    WHERE au.country = 'US' AND au.state = 'AL';
+  ```
+
+#### Show Partitions
+
+  This command gets the Hive partition information of the table
+
+  ```
+  SHOW PARTITIONS [db_name.]table_name
+  ```
+
+#### Drop Partition
+
+  This command drops the specified Hive partition only.
+  ```
+  ALTER TABLE table_name DROP [IF EXISTS] (PARTITION part_spec, ...)
+  ```
+
+#### Insert OVERWRITE
+  
+  This command allows you to insert or load overwrite on a spcific partition.
+  
+  ```
+   INSERT OVERWRITE TABLE table_name
+    PARTITION (column = 'partition_name')
+    select_statement
+  ```
+  
+  Example:
+  ```
+  INSERT OVERWRITE TABLE partitioned_user
+    PARTITION (country = 'US')
+    SELECT * FROM another_user au 
+    WHERE au.country = 'US';
+  ```
+
+### CARBONDATA PARTITION(HASH,RANGE,LIST) -- Alpha feature, this partition not supports update and delete data.
+
+  The partition supports three type:(Hash,Range,List), similar to other system's partition features, CarbonData's partition feature can be used to improve query performance by filtering on the partition column.
+
+### Create Hash Partition Table
+
+  This command allows us to create hash partition.
+  
+  ```
+  CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
+                    [(col_name data_type , ...)]
+  PARTITIONED BY (partition_col_name data_type)
+  STORED BY 'carbondata'
+  [TBLPROPERTIES ('PARTITION_TYPE'='HASH',
+                  'NUM_PARTITIONS'='N' ...)]
+  ```
+  NOTE: N is the number of hash partitions
+
+
+  Example:
+  ```
+  CREATE TABLE IF NOT EXISTS hash_partition_table(
+      col_A String,
+      col_B Int,
+      col_C Long,
+      col_D Decimal(10,2),
+      col_F Timestamp
+  ) PARTITIONED BY (col_E Long)
+  STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='9')
+  ```
+
+### Create Range Partition Table
+
+  This command allows us to create range partition.
+  ```
+  CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
+                    [(col_name data_type , ...)]
+  PARTITIONED BY (partition_col_name data_type)
+  STORED BY 'carbondata'
+  [TBLPROPERTIES ('PARTITION_TYPE'='RANGE',
+                  'RANGE_INFO'='2014-01-01, 2015-01-01, 2016-01-01, ...')]
+  ```
+
+  NOTE:
+  * The 'RANGE_INFO' must be defined in ascending order in the table properties.
+  * The default format for partition column of Date/Timestamp type is yyyy-MM-dd. Alternate formats for Date/Timestamp could be defined in CarbonProperties.
+
+  Example:
+  ```
+  CREATE TABLE IF NOT EXISTS range_partition_table(
+      col_A String,
+      col_B Int,
+      col_C Long,
+      col_D Decimal(10,2),
+      col_E Long
+   ) partitioned by (col_F Timestamp)
+   PARTITIONED BY 'carbondata'
+   TBLPROPERTIES('PARTITION_TYPE'='RANGE',
+   'RANGE_INFO'='2015-01-01, 2016-01-01, 2017-01-01, 2017-02-01')
+  ```
+
+### Create List Partition Table
+
+  This command allows us to create list partition.
+  ```
+  CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
+                    [(col_name data_type , ...)]
+  PARTITIONED BY (partition_col_name data_type)
+  STORED BY 'carbondata'
+  [TBLPROPERTIES ('PARTITION_TYPE'='LIST',
+                  'LIST_INFO'='A, B, C, ...')]
+  ```
+  NOTE: List partition supports list info in one level group.
+
+  Example:
+  ```
+  CREATE TABLE IF NOT EXISTS list_partition_table(
+      col_B Int,
+      col_C Long,
+      col_D Decimal(10,2),
+      col_E Long,
+      col_F Timestamp
+   ) PARTITIONED BY (col_A String)
+   STORED BY 'carbondata'
+   TBLPROPERTIES('PARTITION_TYPE'='LIST',
+   'LIST_INFO'='aaaa, bbbb, (cccc, dddd), eeee')
+  ```
+
+
+### Show Partitions
+
+  The following command is executed to get the partition information of the table
+
+  ```
+  SHOW PARTITIONS [db_name.]table_name
+  ```
+
+### Add a new partition
+
+  ```
+  ALTER TABLE [db_name].table_name ADD PARTITION('new_partition')
+  ```
+
+### Split a partition
+
+  ```
+  ALTER TABLE [db_name].table_name SPLIT PARTITION(partition_id) INTO('new_partition1', 'new_partition2'...)
+  ```
+
+### Drop a partition
+
+   Only drop partition definition, but keep data
+  ```
+    ALTER TABLE [db_name].table_name DROP PARTITION(partition_id)
+   ```
+
+  Drop both partition definition and data
+  ```
+  ALTER TABLE [db_name].table_name DROP PARTITION(partition_id) WITH DATA
+  ```
+
+  NOTE:
+  * Hash partition table is not supported for ADD, SPLIT and DROP commands.
+  * Partition Id: in CarbonData like the hive, folders are not used to divide partitions instead partition id is used to replace the task id. It could make use of the characteristic and meanwhile reduce some metadata.
+
+  ```
+  SegmentDir/0_batchno0-0-1502703086921.carbonindex
+            ^
+  SegmentDir/part-0-0_batchno0-0-1502703086921.carbondata
+                     ^
+  ```
+
+  Here are some useful tips to improve query performance of carbonData partition table:
+  * The partitioned column can be excluded from SORT_COLUMNS, this will let other columns to do the efficient sorting.
+  * When writing SQL on a partition table, try to use filters on the partition column.
+
+
+## PRE-AGGREGATE TABLES
+  Carbondata supports pre aggregating of data so that OLAP kind of queries can fetch data 
+  much faster.Aggregate tables are created as datamaps so that the handling is as efficient as 
+  other indexing support.Users can create as many aggregate tables they require as datamaps to 
+  improve their query performance,provided the storage requirements and loading speeds are 
+  acceptable.
+  
+  For main table called **sales** which is defined as 
+  
+  ```
+  CREATE TABLE sales (
+  order_time timestamp,
+  user_id string,
+  sex string,
+  country string,
+  quantity int,
+  price bigint)
+  STORED BY 'carbondata'
+  ```
+  
+  user can create pre-aggregate tables using the DDL
+  
+  ```
+  CREATE DATAMAP agg_sales
+  ON TABLE sales
+  USING "preaggregate"
+  AS
+  SELECT country, sex, sum(quantity), avg(price)
+  FROM sales
+  GROUP BY country, sex
+  ```
+  
+<b><p align="left">Functions supported in pre-aggregate tables</p></b>
+
+| Function | Rollup supported |
+|-----------|----------------|
+| SUM | Yes |
+| AVG | Yes |
+| MAX | Yes |
+| MIN | Yes |
+| COUNT | Yes |
+
+
+##### How pre-aggregate tables are selected
+For the main table **sales** and pre-aggregate table **agg_sales** created above, queries of the 
+kind
+```
+SELECT country, sex, sum(quantity), avg(price) from sales GROUP BY country, sex
+
+SELECT sex, sum(quantity) from sales GROUP BY sex
+
+SELECT sum(price), country from sales GROUP BY country
+``` 
+
+will be transformed by Query Planner to fetch data from pre-aggregate table **agg_sales**
+
+But queries of kind
+```
+SELECT user_id, country, sex, sum(quantity), avg(price) from sales GROUP BY country, sex
+
+SELECT sex, avg(quantity) from sales GROUP BY sex
+
+SELECT max(price), country from sales GROUP BY country
+```
+
+will fetch the data from the main table **sales**
+
+##### Loading data to pre-aggregate tables
+For existing table with loaded data, data load to pre-aggregate table will be triggered by the 
+CREATE DATAMAP statement when user creates the pre-aggregate table.
+For incremental loads after aggregates tables are created, loading data to main table triggers 
+the load to pre-aggregate tables once main table loading is complete.These loads are automic 
+meaning that data on main table and aggregate tables are only visible to the user after all tables 
+are loaded
+
+##### Querying data from pre-aggregate tables
+Pre-aggregate tables cannot be queries directly.Queries are to be made on main table.Internally 
+carbondata will check associated pre-aggregate tables with the main table and if the 
+pre-aggregate tables satisfy the query condition, the plan is transformed automatically to use 
+pre-aggregate table to fetch the data
+
+##### Compacting pre-aggregate tables
+Compaction is an optional operation for pre-aggregate table. If compaction is performed on main 
+table but not performed on pre-aggregate table, all queries still can benefit from pre-aggregate 
+table.To further improve performance on pre-aggregate table, compaction can be triggered on 
+pre-aggregate tables directly, it will merge the segments inside pre-aggregation table. 
+To do that, use ALTER TABLE COMPACT command on the pre-aggregate table just like the main table
+
+  NOTE:
+  * If the aggregate function used in the pre-aggregate table creation included distinct-count,
+     during compaction, the pre-aggregate table values are recomputed.This would a costly 
+     operation as compared to the compaction of pre-aggregate tables containing other aggregate 
+     functions alone
+ 
+##### Update/Delete Operations on pre-aggregate tables
+This functionality is not supported.
+
+  NOTE (<b>RESTRICTION</b>):
+  * Update/Delete operations are <b>not supported</b> on main table which has pre-aggregate tables 
+  created on it.All the pre-aggregate tables <b>will have to be dropped</b> before update/delete 
+  operations can be performed on the main table.Pre-aggregate tables can be rebuilt manually 
+  after update/delete operations are completed
+ 
+##### Delete Segment Operations on pre-aggregate tables
+This functionality is not supported.
+
+  NOTE (<b>RESTRICTION</b>):
+  * Delete Segment operations are <b>not supported</b> on main table which has pre-aggregate tables 
+  created on it.All the pre-aggregate tables <b>will have to be dropped</b> before update/delete 
+  operations can be performed on the main table.Pre-aggregate tables can be rebuilt manually 
+  after delete segment operations are completed
+  
+##### Alter Table Operations on pre-aggregate tables
+This functionality is not supported.
+
+  NOTE (<b>RESTRICTION</b>):
+  * Adding new column in new table does not have any affect on pre-aggregate tables. However if 
+  dropping or renaming a column has impact in pre-aggregate table, such operations will be 
+  rejected and error will be thrown.All the pre-aggregate tables <b>will have to be dropped</b> 
+  before Alter Operations can be performed on the main table.Pre-aggregate tables can be rebuilt 
+  manually after Alter Table operations are completed
+  
+### Supporting timeseries data (Alpha feature in 1.3.0)
+Carbondata has built-in understanding of time hierarchy and levels: year, month, day, hour, minute.
+Multiple pre-aggregate tables can be created for the hierarchy and Carbondata can do automatic 
+roll-up for the queries on these hierarchies.
+
+  ```
+  CREATE DATAMAP agg_year
+  ON TABLE sales
+  USING "timeseries"
+  DMPROPERTIES (
+  'event_time’=’order_time’,
+  'year_granualrity’=’1’,
+  ) AS
+  SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
+   avg(price) FROM sales GROUP BY order_time, country, sex
+    
+  CREATE DATAMAP agg_month
+  ON TABLE sales
+  USING "timeseries"
+  DMPROPERTIES (
+  'event_time’=’order_time’,
+  'month_granualrity’=’1’,
+  ) AS
+  SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
+   avg(price) FROM sales GROUP BY order_time, country, sex
+    
+  CREATE DATAMAP agg_day
+  ON TABLE sales
+  USING "timeseries"
+  DMPROPERTIES (
+  'event_time’=’order_time’,
+  'day_granualrity’=’1’,
+  ) AS
+  SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
+   avg(price) FROM sales GROUP BY order_time, country, sex
+        
+  CREATE DATAMAP agg_sales_hour
+  ON TABLE sales
+  USING "timeseries"
+  DMPROPERTIES (
+  'event_time’=’order_time’,
+  'hour_granualrity’=’1’,
+  ) AS
+  SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
+   avg(price) FROM sales GROUP BY order_time, country, sex
+  
+  CREATE DATAMAP agg_minute
+  ON TABLE sales
+  USING "timeseries"
+  DMPROPERTIES (
+  'event_time’=’order_time’,
+  'minute_granualrity’=’1’,
+  ) AS
+  SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
+   avg(price) FROM sales GROUP BY order_time, country, sex
+    
+  CREATE DATAMAP agg_minute
+  ON TABLE sales
+  USING "timeseries"
+  DMPROPERTIES (
+  'event_time’=’order_time’,
+  'minute_granualrity’=’1’,
+  ) AS
+  SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
+   avg(price) FROM sales GROUP BY order_time, country, sex
+  ```
+  
+  For Querying data and automatically roll-up to the desired aggregation level,Carbondata supports 
+  UDF as
+  ```
+  timeseries(timeseries column name, ‘aggregation level’)
+  ```
+  ```
+  Select timeseries(order_time, ‘hour’), sum(quantity) from sales group by timeseries(order_time,
+  ’hour’)
+  ```
+  
+  It is **not necessary** to create pre-aggregate tables for each granularity unless required for 
+  query
+  .Carbondata
+   can roll-up the data and fetch it
+   
+  For Example: For main table **sales** , If pre-aggregate tables were created as  
+  
+  ```
+  CREATE DATAMAP agg_day
+    ON TABLE sales
+    USING "timeseries"
+    DMPROPERTIES (
+    'event_time’=’order_time’,
+    'day_granualrity’=’1’,
+    ) AS
+    SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
+     avg(price) FROM sales GROUP BY order_time, country, sex
+          
+    CREATE DATAMAP agg_sales_hour
+    ON TABLE sales
+    USING "timeseries"
+    DMPROPERTIES (
+    'event_time’=’order_time’,
+    'hour_granualrity’=’1’,
+    ) AS
+    SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
+     avg(price) FROM sales GROUP BY order_time, country, sex
+  ```
+  
+  Queries like below will be rolled-up and fetched from pre-aggregate tables
+  ```
+  Select timeseries(order_time, ‘month’), sum(quantity) from sales group by timeseries(order_time,
+    ’month’)
+    
+  Select timeseries(order_time, ‘year’), sum(quantity) from sales group by timeseries(order_time,
+    ’year’)
+  ```
+  
+  NOTE (<b>RESTRICTION</b>):
+  * Only value of 1 is supported for hierarchy levels. Other hierarchy levels are not supported. 
+  Other hierarchy levels are not supported
+  * pre-aggregate tables for the desired levels needs to be created one after the other
+  * pre-aggregate tables created for each level needs to be dropped separately 
+    
+
+## BUCKETING
+
+  Bucketing feature can be used to distribute/organize the table/partition data into multiple files such
+  that similar records are present in the same file. While creating a table, user needs to specify the
+  columns to be used for bucketing and the number of buckets. For the selection of bucket the Hash value
+  of columns is used.
+
+  ```
+  CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
+                    [(col_name data_type, ...)]
+  STORED BY 'carbondata'
+  TBLPROPERTIES('BUCKETNUMBER'='noOfBuckets',
+  'BUCKETCOLUMNS'='columnname')
+  ```
+
+  NOTE:
+  * Bucketing can not be performed for columns of Complex Data Types.
+  * Columns in the BUCKETCOLUMN parameter must be only dimension. The BUCKETCOLUMN parameter can not be a measure or a combination of measures and dimensions.
+
+  Example:
+  ```
+  CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
+                                productNumber Int,
+                                saleQuantity Int,
+                                productName String,
+                                storeCity String,
+                                storeProvince String,
+                                productCategory String,
+                                productBatch String,
+                                revenue Int)
+  STORED BY 'carbondata'
+  TBLPROPERTIES ('BUCKETNUMBER'='4', 'BUCKETCOLUMNS'='productName')
+  ```
+  
+## SEGMENT MANAGEMENT  
+
+### SHOW SEGMENT
+
+  This command is used to get the segments of CarbonData table.
+
+  ```
+  SHOW SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
+  ```
+  
+  Example:
+  ```
+  SHOW SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4
+  ```
+
+### DELETE SEGMENT BY ID
+
+  This command is used to delete segment by using the segment ID. Each segment has a unique segment ID associated with it. 
+  Using this segment ID, you can remove the segment.
+
+  The following command will get the segmentID.
+
+  ```
+  SHOW SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
+  ```
+
+  After you retrieve the segment ID of the segment that you want to delete, execute the following command to delete the selected segment.
+
+  ```
+  DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.ID IN (segment_id1, segments_id2, ...)
+  ```
+
+  Example:
+
+  ```
+  DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0)
+  DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0,5,8)
+  ```
+
+### DELETE SEGMENT BY DATE
+
+  This command will allow to delete the CarbonData segment(s) from the store based on the date provided by the user in the DML command. 
+  The segment created before the particular date will be removed from the specific stores.
+
+  ```
+  DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.STARTTIME BEFORE DATE_VALUE
+  ```
+
+  Example:
+  ```
+  DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.STARTTIME BEFORE '2017-06-01 12:05:06' 
+  ```
+
+### QUERY DATA WITH SPECIFIED SEGMENTS
+
+  This command is used to read data from specified segments during CarbonScan.
+  
+  Get the Segment ID:
+  ```
+  SHOW SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
+  ```
+  
+  Set the segment IDs for table
+  ```
+  SET carbon.input.segments.<database_name>.<table_name> = <list of segment IDs>
+  ```
+  
+  NOTE:
+  carbon.input.segments: Specifies the segment IDs to be queried. This property allows you to query specified segments of the specified table. The CarbonScan will read data from specified segments only.
+  
+  If user wants to query with segments reading in multi threading mode, then CarbonSession.threadSet can be used instead of SET query.
+  ```
+  CarbonSession.threadSet ("carbon.input.segments.<database_name>.<table_name>","<list of segment IDs>");
+  ```
+  
+  Reset the segment IDs
+  ```
+  SET carbon.input.segments.<database_name>.<table_name> = *;
+  ```
+  
+  If user wants to query with segments reading in multi threading mode, then CarbonSession.threadSet can be used instead of SET query. 
+  ```
+  CarbonSession.threadSet ("carbon.input.segments.<database_name>.<table_name>","*");
+  ```
+  
+  **Examples:**
+  
+  * Example to show the list of segment IDs,segment status, and other required details and then specify the list of segments to be read.
+  
+  ```
+  SHOW SEGMENTS FOR carbontable1;
+  
+  SET carbon.input.segments.db.carbontable1 = 1,3,9;
+  ```
+  
+  * Example to query with segments reading in multi threading mode:
+  
+  ```
+  CarbonSession.threadSet ("carbon.input.segments.db.carbontable_Multi_Thread","1,3");
+  ```
+  
+  * Example for threadset in multithread environment (following shows how it is used in Scala code):
+  
+  ```
+  def main(args: Array[String]) {
+  Future {          
+    CarbonSession.threadSet ("carbon.input.segments.db.carbontable_Multi_Thread","1")
+    spark.sql("select count(empno) from carbon.input.segments.db.carbontable_Multi_Thread").show();
+     }
+   }
+  ```

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/711502d1/src/site/markdown/faq.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/faq.md b/src/site/markdown/faq.md
new file mode 100644
index 0000000..baa46cc
--- /dev/null
+++ b/src/site/markdown/faq.md
@@ -0,0 +1,181 @@
+<!--
+    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.
+-->
+
+# FAQs
+
+* [What are Bad Records?](#what-are-bad-records)
+* [Where are Bad Records Stored in CarbonData?](#where-are-bad-records-stored-in-carbondata)
+* [How to enable Bad Record Logging?](#how-to-enable-bad-record-logging)
+* [How to ignore the Bad Records?](#how-to-ignore-the-bad-records)
+* [How to specify store location while creating carbon session?](#how-to-specify-store-location-while-creating-carbon-session)
+* [What is Carbon Lock Type?](#what-is-carbon-lock-type)
+* [How to resolve Abstract Method Error?](#how-to-resolve-abstract-method-error)
+* [How Carbon will behave when execute insert operation in abnormal scenarios?](#how-carbon-will-behave-when-execute-insert-operation-in-abnormal-scenarios)
+* [Why aggregate query is not fetching data from aggregate table?] (#why-aggregate-query-is-not-fetching-data-from-aggregate-table)
+
+## What are Bad Records?
+Records that fail to get loaded into the CarbonData due to data type incompatibility or are empty or have incompatible format are classified as Bad Records.
+
+## Where are Bad Records Stored in CarbonData?
+The bad records are stored at the location set in carbon.badRecords.location in carbon.properties file.
+By default **carbon.badRecords.location** specifies the following location ``/opt/Carbon/Spark/badrecords``.
+
+## How to enable Bad Record Logging?
+While loading data we can specify the approach to handle Bad Records. In order to analyse the cause of the Bad Records the parameter ``BAD_RECORDS_LOGGER_ENABLE`` must be set to value ``TRUE``. There are multiple approaches to handle Bad Records which can be specified  by the parameter ``BAD_RECORDS_ACTION``.
+
+- To pad the incorrect values of the csv rows with NULL value and load the data in CarbonData, set the following in the query :
+```
+'BAD_RECORDS_ACTION'='FORCE'
+```
+
+- To write the Bad Records without padding incorrect values with NULL in the raw csv (set in the parameter **carbon.badRecords.location**), set the following in the query :
+```
+'BAD_RECORDS_ACTION'='REDIRECT'
+```
+
+## How to ignore the Bad Records?
+To ignore the Bad Records from getting stored in the raw csv, we need to set the following in the query :
+```
+'BAD_RECORDS_ACTION'='IGNORE'
+```
+
+## How to specify store location while creating carbon session?
+The store location specified while creating carbon session is used by the CarbonData to store the meta data like the schema, dictionary files, dictionary meta data and sort indexes.
+
+Try creating ``carbonsession`` with ``storepath`` specified in the following manner :
+
+```
+val carbon = SparkSession.builder().config(sc.getConf)
+             .getOrCreateCarbonSession(<store_path>)
+```
+Example:
+
+```
+val carbon = SparkSession.builder().config(sc.getConf)
+             .getOrCreateCarbonSession("hdfs://localhost:9000/carbon/store")
+```
+
+## What is Carbon Lock Type?
+The Apache CarbonData acquires lock on the files to prevent concurrent operation from modifying the same files. The lock can be of the following types depending on the storage location, for HDFS we specify it to be of type HDFSLOCK. By default it is set to type LOCALLOCK.
+The property carbon.lock.type configuration specifies the type of lock to be acquired during concurrent operations on table. This property can be set with the following values :
+- **LOCALLOCK** : This Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently.
+- **HDFSLOCK** : This Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData spark applications are launched and no ZooKeeper is running on cluster and the HDFS supports, file based locking.
+
+## How to resolve Abstract Method Error?
+In order to build CarbonData project it is necessary to specify the spark profile. The spark profile sets the Spark Version. You need to specify the ``spark version`` while using Maven to build project.
+
+## How Carbon will behave when execute insert operation in abnormal scenarios?
+Carbon support insert operation, you can refer to the syntax mentioned in [DML Operations on CarbonData](dml-operation-on-carbondata.md).
+First, create a soucre table in spark-sql and load data into this created table.
+
+```
+CREATE TABLE source_table(
+id String,
+name String,
+city String)
+ROW FORMAT DELIMITED FIELDS TERMINATED BY ",";
+```
+
+```
+SELECT * FROM source_table;
+id  name    city
+1   jack    beijing
+2   erlu    hangzhou
+3   davi    shenzhen
+```
+
+**Scenario 1** :
+
+Suppose, the column order in carbon table is different from source table, use script "SELECT * FROM carbon table" to query, will get the column order similar as source table, rather than in carbon table's column order as expected. 
+
+```
+CREATE TABLE IF NOT EXISTS carbon_table(
+id String,
+city String,
+name String)
+STORED BY 'carbondata';
+```
+
+```
+INSERT INTO TABLE carbon_table SELECT * FROM source_table;
+```
+
+```
+SELECT * FROM carbon_table;
+id  city    name
+1   jack    beijing
+2   erlu    hangzhou
+3   davi    shenzhen
+```
+
+As result shows, the second column is city in carbon table, but what inside is name, such as jack. This phenomenon is same with insert data into hive table.
+
+If you want to insert data into corresponding column in carbon table, you have to specify the column order same in insert statment. 
+
+```
+INSERT INTO TABLE carbon_table SELECT id, city, name FROM source_table;
+```
+
+**Scenario 2** :
+
+Insert operation will be failed when the number of column in carbon table is different from the column specified in select statement. The following insert operation will be failed.
+
+```
+INSERT INTO TABLE carbon_table SELECT id, city FROM source_table;
+```
+
+**Scenario 3** :
+
+When the column type in carbon table is different from the column specified in select statement. The insert operation will still success, but you may get NULL in result, because NULL will be substitute value when conversion type failed.
+
+## Why aggregate query is not fetching data from aggregate table?
+Following are the aggregate queries that won’t fetch data from aggregate table:
+
+- **Scenario 1** :
+When SubQuery predicate is present in the query.
+
+Example 
+
+```
+create table gdp21(cntry smallint, gdp double, y_year date) stored by 'carbondata'
+create datamap ag1 on table gdp21 using 'preaggregate' as select cntry, sum(gdp) from gdp group by ctry;
+select ctry from pop1 where ctry in (select cntry from gdp21 group by cntry)
+```
+
+- **Scenario 2** : 
+When aggregate function along with ‘in’ filter. 
+
+Example.
+
+```
+create table gdp21(cntry smallint, gdp double, y_year date) stored by 'carbondata'
+create datamap ag1 on table gdp21 using 'preaggregate' as select cntry, sum(gdp) from gdp group by ctry;
+select cntry, sum(gdp) from gdp21 where cntry in (select ctry from pop1) group by cntry;
+```
+
+- **Scenario 3** : 
+When aggregate function having ‘join’ with Equal filter.
+
+Example.
+
+```
+create table gdp21(cntry smallint, gdp double, y_year date) stored by 'carbondata'
+create datamap ag1 on table gdp21 using 'preaggregate' as select cntry, sum(gdp) from gdp group by ctry;
+select cntry,sum(gdp) from gdp21,pop1 where cntry=ctry group by cntry;
+```
+
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/711502d1/src/site/markdown/file-structure-of-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/file-structure-of-carbondata.md b/src/site/markdown/file-structure-of-carbondata.md
new file mode 100644
index 0000000..ef31ffd
--- /dev/null
+++ b/src/site/markdown/file-structure-of-carbondata.md
@@ -0,0 +1,40 @@
+<!--
+    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.
+-->
+
+# CarbonData File Structure
+
+CarbonData files contain groups of data called blocklets, along with all required information like schema, offsets and indices etc, in a file header and footer, co-located in HDFS.
+
+The file footer can be read once to build the indices in memory, which can be utilized for optimizing the scans and processing for all subsequent queries.
+
+### Understanding CarbonData File Structure
+* Block : It would be as same as HDFS block, CarbonData creates one file for each data block, user can specify TABLE_BLOCKSIZE during creation table. Each file contains File Header, Blocklets and File Footer.
+
+![CarbonData File Structure](../../src/site/images/carbon_data_file_structure_new.png)
+
+* File Header : It contains CarbonData file version number, list of column schema and schema updation timestamp.
+* File Footer : it contains Number of rows, segmentinfo ,all blocklets’ info and index, you can find the detail from the below diagram.
+* Blocklet : Rows are grouped to form a blocklet, the size of the blocklet is configurable and default size is 64MB, Blocklet contains Column Page groups for each column.
+* Column Page Group : Data of one column and it is further divided into pages, it is guaranteed to be contiguous in file.
+* Page : It has the data of one column and the number of row is fixed to 32000 size.
+
+![CarbonData File Format](../../src/site/images/carbon_data_format_new.png)
+
+### Each page contains three types of data
+* Data Page: Contains the encoded data of a column of columns.
+* Row ID Page (optional): Contains the row ID mappings used when the data page is stored as an inverted index.
+* RLE Page (optional): Contains additional metadata used when the data page is RLE coded.

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/711502d1/src/site/markdown/installation-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/installation-guide.md b/src/site/markdown/installation-guide.md
new file mode 100644
index 0000000..1ba5dd1
--- /dev/null
+++ b/src/site/markdown/installation-guide.md
@@ -0,0 +1,189 @@
+<!--
+    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.
+-->
+
+# Installation Guide
+This tutorial guides you through the installation and configuration of CarbonData in the following two modes :
+
+* [Installing and Configuring CarbonData on Standalone Spark Cluster](#installing-and-configuring-carbondata-on-standalone-spark-cluster)
+* [Installing and Configuring CarbonData on Spark on YARN Cluster](#installing-and-configuring-carbondata-on-spark-on-yarn-cluster)
+
+followed by :
+
+* [Query Execution using CarbonData Thrift Server](#query-execution-using-carbondata-thrift-server)
+
+## Installing and Configuring CarbonData on Standalone Spark Cluster
+
+### Prerequisites
+
+   - Hadoop HDFS and Yarn should be installed and running.
+
+   - Spark should be installed and running on all the cluster nodes.
+
+   - CarbonData user should have permission to access HDFS.
+
+
+### Procedure
+
+1. [Build the CarbonData](https://github.com/apache/carbondata/blob/master/build/README.md) project and get the assembly jar from `./assembly/target/scala-2.1x/carbondata_xxx.jar`. 
+
+2. Copy `./assembly/target/scala-2.1x/carbondata_xxx.jar` to `$SPARK_HOME/carbonlib` folder.
+
+     **NOTE**: Create the carbonlib folder if it does not exist inside `$SPARK_HOME` path.
+
+3. Add the carbonlib folder path in the Spark classpath. (Edit `$SPARK_HOME/conf/spark-env.sh` file and modify the value of `SPARK_CLASSPATH` by appending `$SPARK_HOME/carbonlib/*` to the existing value)
+
+4. Copy the `./conf/carbon.properties.template` file from CarbonData repository to `$SPARK_HOME/conf/` folder and rename the file to `carbon.properties`.
+
+5. Repeat Step 2 to Step 5 in all the nodes of the cluster.
+    
+6. In Spark node[master], configure the properties mentioned in the following table in `$SPARK_HOME/conf/spark-defaults.conf` file.
+
+| Property | Value | Description |
+|---------------------------------|-----------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------|
+| spark.driver.extraJavaOptions | `-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties` | A string of extra JVM options to pass to the driver. For instance, GC settings or other logging. |
+| spark.executor.extraJavaOptions | `-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties` | A string of extra JVM options to pass to executors. For instance, GC settings or other logging. **NOTE**: You can enter multiple values separated by space. |
+
+7. Add the following properties in `$SPARK_HOME/conf/carbon.properties` file:
+
+| Property             | Required | Description                                                                            | Example                             | Remark  |
+|----------------------|----------|----------------------------------------------------------------------------------------|-------------------------------------|---------|
+| carbon.storelocation | NO       | Location where data CarbonData will create the store and write the data in its own format. | hdfs://HOSTNAME:PORT/Opt/CarbonStore      | Propose to set HDFS directory |
+
+
+8. Verify the installation. For example:
+
+```
+./spark-shell --master spark://HOSTNAME:PORT --total-executor-cores 2
+--executor-memory 2G
+```
+
+**NOTE**: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.
+
+To get started with CarbonData : [Quick Start](quick-start-guide.md), [DDL Operations on CarbonData](ddl-operation-on-carbondata.md)
+
+## Installing and Configuring CarbonData on Spark on YARN Cluster
+
+   This section provides the procedure to install CarbonData on "Spark on YARN" cluster.
+
+### Prerequisites
+   * Hadoop HDFS and Yarn should be installed and running.
+   * Spark should be installed and running in all the clients.
+   * CarbonData user should have permission to access HDFS.
+
+### Procedure
+
+   The following steps are only for Driver Nodes. (Driver nodes are the one which starts the spark context.)
+
+1. [Build the CarbonData](https://github.com/apache/carbondata/blob/master/build/README.md) project and get the assembly jar from `./assembly/target/scala-2.1x/carbondata_xxx.jar` and copy to `$SPARK_HOME/carbonlib` folder.
+
+    **NOTE**: Create the carbonlib folder if it does not exists inside `$SPARK_HOME` path.
+
+2. Copy the `./conf/carbon.properties.template` file from CarbonData repository to `$SPARK_HOME/conf/` folder and rename the file to `carbon.properties`.
+
+3. Create `tar.gz` file of carbonlib folder and move it inside the carbonlib folder.
+
+```
+cd $SPARK_HOME
+tar -zcvf carbondata.tar.gz carbonlib/
+mv carbondata.tar.gz carbonlib/
+```
+
+4. Configure the properties mentioned in the following table in `$SPARK_HOME/conf/spark-defaults.conf` file.
+
+| Property | Description | Value |
+|---------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------------------------------|
+| spark.master | Set this value to run the Spark in yarn cluster mode. | Set yarn-client to run the Spark in yarn cluster mode. |
+| spark.yarn.dist.files | Comma-separated list of files to be placed in the working directory of each executor. |`$SPARK_HOME/conf/carbon.properties` |
+| spark.yarn.dist.archives | Comma-separated list of archives to be extracted into the working directory of each executor. |`$SPARK_HOME/carbonlib/carbondata.tar.gz` |
+| spark.executor.extraJavaOptions | A string of extra JVM options to pass to executors. For instance  **NOTE**: You can enter multiple values separated by space. |`-Dcarbon.properties.filepath = carbon.properties` |
+| spark.executor.extraClassPath | Extra classpath entries to prepend to the classpath of executors. **NOTE**: If SPARK_CLASSPATH is defined in spark-env.sh, then comment it and append the values in below parameter spark.driver.extraClassPath |`carbondata.tar.gz/carbonlib/*` |
+| spark.driver.extraClassPath | Extra classpath entries to prepend to the classpath of the driver. **NOTE**: If SPARK_CLASSPATH is defined in spark-env.sh, then comment it and append the value in below parameter spark.driver.extraClassPath. |`$SPARK_HOME/carbonlib/*` |
+| spark.driver.extraJavaOptions | A string of extra JVM options to pass to the driver. For instance, GC settings or other logging. |`-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties` |
+
+
+5. Add the following properties in `$SPARK_HOME/conf/carbon.properties`:
+
+| Property | Required | Description | Example | Default Value |
+|----------------------|----------|----------------------------------------------------------------------------------------|-------------------------------------|---------------|
+| carbon.storelocation | NO | Location where CarbonData will create the store and write the data in its own format. | hdfs://HOSTNAME:PORT/Opt/CarbonStore | Propose to set HDFS directory|
+
+6. Verify the installation.
+
+```
+ ./bin/spark-shell --master yarn-client --driver-memory 1g
+ --executor-cores 2 --executor-memory 2G
+```
+  **NOTE**: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.
+
+  Getting started with CarbonData : [Quick Start](quick-start-guide.md), [DDL Operations on CarbonData](ddl-operation-on-carbondata.md)
+
+## Query Execution Using CarbonData Thrift Server
+
+### Starting CarbonData Thrift Server.
+
+   a. cd `$SPARK_HOME`
+
+   b. Run the following command to start the CarbonData thrift server.
+
+```
+./bin/spark-submit
+--conf spark.sql.hive.thriftServer.singleSession=true
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer
+$SPARK_HOME/carbonlib/$CARBON_ASSEMBLY_JAR <carbon_store_path>
+```
+
+| Parameter | Description | Example |
+|---------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------|
+| CARBON_ASSEMBLY_JAR | CarbonData assembly jar name present in the `$SPARK_HOME/carbonlib/` folder. | carbondata_2.xx-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar |
+| carbon_store_path | This is a parameter to the CarbonThriftServer class. This a HDFS path where CarbonData files will be kept. Strongly Recommended to put same as carbon.storelocation parameter of carbon.properties. | `hdfs://<host_name>:port/user/hive/warehouse/carbon.store` |
+
+**Examples**
+   
+   * Start with default memory and executors.
+
+```
+./bin/spark-submit
+--conf spark.sql.hive.thriftServer.singleSession=true
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer 
+$SPARK_HOME/carbonlib
+/carbondata_2.xx-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar
+hdfs://<host_name>:port/user/hive/warehouse/carbon.store
+```
+   
+   * Start with Fixed executors and resources.
+
+```
+./bin/spark-submit --conf spark.sql.hive.thriftServer.singleSession=true 
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer 
+--num-executors 3 --driver-memory 20g --executor-memory 250g 
+--executor-cores 32 
+/srv/OSCON/BigData/HACluster/install/spark/sparkJdbc/lib
+/carbondata_2.xx-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar
+hdfs://<host_name>:port/user/hive/warehouse/carbon.store
+```
+  
+### Connecting to CarbonData Thrift Server Using Beeline.
+
+```
+     cd $SPARK_HOME
+     ./sbin/start-thriftserver.sh
+     ./bin/beeline -u jdbc:hive2://<thriftserver_host>:port
+
+     Example
+     ./bin/beeline -u jdbc:hive2://10.10.10.10:10000
+```
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/711502d1/src/site/markdown/quick-start-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/quick-start-guide.md b/src/site/markdown/quick-start-guide.md
new file mode 100644
index 0000000..84f871d
--- /dev/null
+++ b/src/site/markdown/quick-start-guide.md
@@ -0,0 +1,99 @@
+<!--
+    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.
+-->
+
+# Quick Start
+This tutorial provides a quick introduction to using CarbonData.
+
+##  Prerequisites
+* [Installation and building CarbonData](https://github.com/apache/carbondata/blob/master/build).
+* Create a sample.csv file using the following commands. The CSV file is required for loading data into CarbonData.
+
+  ```
+  cd carbondata
+  cat > sample.csv << EOF
+  id,name,city,age
+  1,david,shenzhen,31
+  2,eason,shenzhen,27
+  3,jarry,wuhan,35
+  EOF
+  ```
+
+## Interactive Analysis with Spark Shell Version 2.1
+
+Apache Spark Shell provides a simple way to learn the API, as well as a powerful tool to analyze data interactively. Please visit [Apache Spark Documentation](http://spark.apache.org/docs/latest/) for more details on Spark shell.
+
+#### Basics
+
+Start Spark shell by running the following command in the Spark directory:
+
+```
+./bin/spark-shell --jars <carbondata assembly jar path>
+```
+**NOTE**: Assembly jar will be available after [building CarbonData](https://github.com/apache/carbondata/blob/master/build/README.md) and can be copied from `./assembly/target/scala-2.1x/carbondata_xxx.jar`
+
+In this shell, SparkSession is readily available as `spark` and Spark context is readily available as `sc`.
+
+In order to create a CarbonSession we will have to configure it explicitly in the following manner :
+
+* Import the following :
+
+```
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.CarbonSession._
+```
+
+* Create a CarbonSession :
+
+```
+val carbon = SparkSession.builder().config(sc.getConf)
+             .getOrCreateCarbonSession("<hdfs store path>")
+```
+**NOTE**: By default metastore location is pointed to `../carbon.metastore`, user can provide own metastore location to CarbonSession like `SparkSession.builder().config(sc.getConf)
+.getOrCreateCarbonSession("<hdfs store path>", "<local metastore path>")`
+
+#### Executing Queries
+
+###### Creating a Table
+
+```
+scala>carbon.sql("CREATE TABLE
+                        IF NOT EXISTS test_table(
+                                  id string,
+                                  name string,
+                                  city string,
+                                  age Int)
+                       STORED BY 'carbondata'")
+```
+
+###### Loading Data to a Table
+
+```
+scala>carbon.sql("LOAD DATA INPATH '/path/to/sample.csv'
+                  INTO TABLE test_table")
+```
+**NOTE**: Please provide the real file path of `sample.csv` for the above script. 
+If you get "tablestatus.lock" issue, please refer to [troubleshooting](troubleshooting.md)
+
+###### Query Data from a Table
+
+```
+scala>carbon.sql("SELECT * FROM test_table").show()
+
+scala>carbon.sql("SELECT city, avg(age), sum(age)
+                  FROM test_table
+                  GROUP BY city").show()
+```

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/711502d1/src/site/markdown/supported-data-types-in-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/supported-data-types-in-carbondata.md b/src/site/markdown/supported-data-types-in-carbondata.md
new file mode 100644
index 0000000..6c21508
--- /dev/null
+++ b/src/site/markdown/supported-data-types-in-carbondata.md
@@ -0,0 +1,43 @@
+<!--
+    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.
+-->
+
+#  Data Types
+
+#### CarbonData supports the following data types:
+
+  * Numeric Types
+    * SMALLINT
+    * INT/INTEGER
+    * BIGINT
+    * DOUBLE
+    * DECIMAL
+
+  * Date/Time Types
+    * TIMESTAMP
+    * DATE
+
+  * String Types
+    * STRING
+    * CHAR
+    * VARCHAR
+
+  * Complex Types
+    * arrays: ARRAY``<data_type>``
+    * structs: STRUCT``<col_name : data_type COMMENT col_comment, ...>``
+
+  * Other Types
+    * BOOLEAN
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/711502d1/src/site/markdown/troubleshooting.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/troubleshooting.md b/src/site/markdown/troubleshooting.md
new file mode 100644
index 0000000..68dd538
--- /dev/null
+++ b/src/site/markdown/troubleshooting.md
@@ -0,0 +1,267 @@
+<!--
+    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.
+-->
+
+# Troubleshooting
+This tutorial is designed to provide troubleshooting for end users and developers
+who are building, deploying, and using CarbonData.
+
+## When loading data, gets tablestatus.lock issues:
+
+  **Symptom**
+```
+17/11/11 16:48:13 ERROR LocalFileLock: main hdfs:/localhost:9000/carbon/store/default/hdfstable/tablestatus.lock (No such file or directory)
+java.io.FileNotFoundException: hdfs:/localhost:9000/carbon/store/default/hdfstable/tablestatus.lock (No such file or directory)
+	at java.io.FileOutputStream.open0(Native Method)
+	at java.io.FileOutputStream.open(FileOutputStream.java:270)
+	at java.io.FileOutputStream.<init>(FileOutputStream.java:213)
+	at java.io.FileOutputStream.<init>(FileOutputStream.java:101)
+```
+
+  **Possible Cause**
+  If you use `<hdfs path>` as store path when creating carbonsession, may get the errors,because the default is LOCALLOCK.
+
+  **Procedure**
+  Before creating carbonsession, sets as below:
+  ```
+  import org.apache.carbondata.core.util.CarbonProperties
+  import org.apache.carbondata.core.constants.CarbonCommonConstants
+  CarbonProperties.getInstance().addProperty(CarbonCommonConstants.LOCK_TYPE, "HDFSLOCK")
+  ```
+
+## Failed to load thrift libraries
+
+  **Symptom**
+
+  Thrift throws following exception :
+
+  ```
+  thrift: error while loading shared libraries:
+  libthriftc.so.0: cannot open shared object file: No such file or directory
+  ```
+
+  **Possible Cause**
+
+  The complete path to the directory containing the libraries is not configured correctly.
+
+  **Procedure**
+
+  Follow the Apache thrift docs at [https://thrift.apache.org/docs/install](https://thrift.apache.org/docs/install) to install thrift correctly.
+
+## Failed to launch the Spark Shell
+
+  **Symptom**
+
+  The shell prompts the following error :
+
+  ```
+  org.apache.spark.sql.CarbonContext$$anon$$apache$spark$sql$catalyst$analysis
+  $OverrideCatalog$_setter_$org$apache$spark$sql$catalyst$analysis
+  $OverrideCatalog$$overrides_$e
+  ```
+
+  **Possible Cause**
+
+  The Spark Version and the selected Spark Profile do not match.
+
+  **Procedure**
+
+  1. Ensure your spark version and selected profile for spark are correct.
+
+  2. Use the following command :
+
+```
+"mvn -Pspark-2.1 -Dspark.version {yourSparkVersion} clean package"
+```
+Note :  Refrain from using "mvn clean package" without specifying the profile.
+
+## Failed to execute load query on cluster.
+
+  **Symptom**
+
+  Load query failed with the following exception:
+
+  ```
+  Dictionary file is locked for updation.
+  ```
+
+  **Possible Cause**
+
+  The carbon.properties file is not identical in all the nodes of the cluster.
+
+  **Procedure**
+
+  Follow the steps to ensure the carbon.properties file is consistent across all the nodes:
+
+  1. Copy the carbon.properties file from the master node to all the other nodes in the cluster.
+     For example, you can use ssh to copy this file to all the nodes.
+
+  2. For the changes to take effect, restart the Spark cluster.
+
+## Failed to execute insert query on cluster.
+
+  **Symptom**
+
+  Load query failed with the following exception:
+
+  ```
+  Dictionary file is locked for updation.
+  ```
+
+  **Possible Cause**
+
+  The carbon.properties file is not identical in all the nodes of the cluster.
+
+  **Procedure**
+
+  Follow the steps to ensure the carbon.properties file is consistent across all the nodes:
+
+  1. Copy the carbon.properties file from the master node to all the other nodes in the cluster.
+       For example, you can use scp to copy this file to all the nodes.
+
+  2. For the changes to take effect, restart the Spark cluster.
+
+## Failed to connect to hiveuser with thrift
+
+  **Symptom**
+
+  We get the following exception :
+
+  ```
+  Cannot connect to hiveuser.
+  ```
+
+  **Possible Cause**
+
+  The external process does not have permission to access.
+
+  **Procedure**
+
+  Ensure that the Hiveuser in mysql must allow its access to the external processes.
+
+## Failed to read the metastore db during table creation.
+
+  **Symptom**
+
+  We get the following exception on trying to connect :
+
+  ```
+  Cannot read the metastore db
+  ```
+
+  **Possible Cause**
+
+  The metastore db is dysfunctional.
+
+  **Procedure**
+
+  Remove the metastore db from the carbon.metastore in the Spark Directory.
+
+## Failed to load data on the cluster
+
+  **Symptom**
+
+  Data loading fails with the following exception :
+
+   ```
+   Data Load failure exeception
+   ```
+
+  **Possible Cause**
+
+  The following issue can cause the failure :
+
+  1. The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.
+
+  2. Path to hdfs ddl is not configured correctly in the carbon.properties.
+
+  **Procedure**
+
+   Follow the steps to ensure the following configuration files are consistent across all the nodes:
+
+   1. Copy the core-site.xml, hive-site.xml, yarn-site,carbon.properties files from the master node to all the other nodes in the cluster.
+      For example, you can use scp to copy this file to all the nodes.
+
+      Note : Set the path to hdfs ddl in carbon.properties in the master node.
+
+   2. For the changes to take effect, restart the Spark cluster.
+
+
+
+## Failed to insert data on the cluster
+
+  **Symptom**
+
+  Insertion fails with the following exception :
+
+   ```
+   Data Load failure exeception
+   ```
+
+  **Possible Cause**
+
+  The following issue can cause the failure :
+
+  1. The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.
+
+  2. Path to hdfs ddl is not configured correctly in the carbon.properties.
+
+  **Procedure**
+
+   Follow the steps to ensure the following configuration files are consistent across all the nodes:
+
+   1. Copy the core-site.xml, hive-site.xml, yarn-site,carbon.properties files from the master node to all the other nodes in the cluster.
+      For example, you can use scp to copy this file to all the nodes.
+
+      Note : Set the path to hdfs ddl in carbon.properties in the master node.
+
+   2. For the changes to take effect, restart the Spark cluster.
+
+## Failed to execute Concurrent Operations(Load,Insert,Update) on table by multiple workers.
+
+  **Symptom**
+
+  Execution fails with the following exception :
+
+   ```
+   Table is locked for updation.
+   ```
+
+  **Possible Cause**
+
+  Concurrency not supported.
+
+  **Procedure**
+
+  Worker must wait for the query execution to complete and the table to release the lock for another query execution to succeed.
+
+## Failed to create a table with a single numeric column.
+
+  **Symptom**
+
+  Execution fails with the following exception :
+
+   ```
+   Table creation fails.
+   ```
+
+  **Possible Cause**
+
+  Behaviour not supported.
+
+  **Procedure**
+
+  A single column that can be considered as dimension is mandatory for table creation.