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/09/07 16:53:48 UTC

[01/39] carbondata-site git commit: Added new page layout & updated as per new md files

Repository: carbondata-site
Updated Branches:
  refs/heads/asf-site 324588f48 -> bee563340


http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/segment-management-on-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/segment-management-on-carbondata.md b/src/site/markdown/segment-management-on-carbondata.md
new file mode 100644
index 0000000..a519c88
--- /dev/null
+++ b/src/site/markdown/segment-management-on-carbondata.md
@@ -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.
+-->
+
+
+## SEGMENT MANAGEMENT
+
+Each load into CarbonData is written into a separate folder called Segment.Segments is a powerful 
+concept which helps to maintain consistency of data and easy transaction management.CarbonData provides DML (Data Manipulation Language) commands to maintain the segments.
+
+- [Show Segments](#show-segment)
+- [Delete Segment by ID](#delete-segment-by-id)
+- [Delete Segment by Date](#delete-segment-by-date)
+- [Query Data with Specified Segments](#query-data-with-specified-segments)
+
+### SHOW SEGMENT
+
+  This command is used to list the segments of CarbonData table.
+
+  ```
+  SHOW [HISTORY] SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
+  ```
+
+  Example:
+  Show visible segments
+  ```
+  SHOW SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4
+  ```
+  Show all segments, include invisible segments
+  ```
+  SHOW HISTORY 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();
+     }
+   }
+  ```
+
+
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/streaming-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/streaming-guide.md b/src/site/markdown/streaming-guide.md
index 32d24dc..2f8aa5e 100644
--- a/src/site/markdown/streaming-guide.md
+++ b/src/site/markdown/streaming-guide.md
@@ -259,3 +259,16 @@ ALTER TABLE streaming_table COMPACT 'close_streaming'
 5. if the table has dictionary columns, it will not support concurrent data loading.
 6. block delete "streaming" segment while the streaming ingestion is running.
 7. block drop the streaming table while the streaming ingestion is running.
+
+
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/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
index eb74a2e..35e41ba 100644
--- a/src/site/markdown/supported-data-types-in-carbondata.md
+++ b/src/site/markdown/supported-data-types-in-carbondata.md
@@ -36,7 +36,7 @@
     * VARCHAR
 
     **NOTE**: For string longer than 32000 characters, use `LONG_STRING_COLUMNS` in table property.
-    Please refer to TBLProperties in [CreateTable](https://github.com/apache/carbondata/blob/master/docs/data-management-on-carbondata.md#create-table) for more information.
+    Please refer to TBLProperties in [CreateTable](./ddl-of-carbondata.md#create-table) for more information.
 
   * Complex Types
     * arrays: ARRAY``<data_type>``
@@ -45,4 +45,16 @@
     **NOTE**: Only 2 level complex type schema is supported for now.
 
   * Other Types
-    * BOOLEAN
\ No newline at end of file
+    * BOOLEAN
+    
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/timeseries-datamap-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/timeseries-datamap-guide.md b/src/site/markdown/timeseries-datamap-guide.md
index 135188d..d3ef3c6 100644
--- a/src/site/markdown/timeseries-datamap-guide.md
+++ b/src/site/markdown/timeseries-datamap-guide.md
@@ -22,18 +22,19 @@
 * [Data Management](#data-management-with-pre-aggregate-tables)
 
 ## Timeseries DataMap Introduction (Alpha Feature)
-Timeseries DataMap a pre-aggregate table implementation based on 'pre-aggregate' DataMap.
+Timeseries DataMap is a pre-aggregate table implementation based on 'pre-aggregate' DataMap.
 Difference is that Timeseries DataMap has built-in understanding of time hierarchy and
 levels: year, month, day, hour, minute, so that it supports automatic roll-up in time dimension 
 for query.
 
+**CAUTION:** Current version of CarbonData does not support roll-up.It will be implemented in future versions.
+
 The data loading, querying, compaction command and its behavior is the same as preaggregate DataMap.
-Please refer to [Pre-aggregate DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/preaggregate-datamap-guide.md)
+Please refer to [Pre-aggregate DataMap](./preaggregate-datamap-guide.md)
 for more information.
   
 To use this datamap, user can create multiple timeseries datamap on the main table which has 
-a *event_time* column, one datamap for one time granularity. Then Carbondata can do automatic 
-roll-up for queries on the main table.
+a *event_time* column, one datamap for one time granularity.
 
 For example, below statement effectively create multiple pre-aggregate tables  on main table called 
 **timeseries**
@@ -88,20 +89,10 @@ DMPROPERTIES (
 ) 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_granularity'='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 timeseries data, Carbondata has builtin support for following time related UDF 
-to enable automatically roll-up to the desired aggregation level
+
 ```
 timeseries(timeseries column name, 'aggregation level')
 ```
@@ -111,7 +102,7 @@ SELECT timeseries(order_time, 'hour'), sum(quantity) FROM sales GROUP BY timeser
 ```
   
 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.
+query.
  
 For Example: For main table **sales** , if following timeseries datamaps were created for day 
 level and hour level pre-aggregate
@@ -138,7 +129,7 @@ level and hour level pre-aggregate
    avg(price) FROM sales GROUP BY order_time, country, sex
 ```
 
-Queries like below will be rolled-up and hit the timeseries datamaps
+Queries like below will not be rolled-up and hit the main table
 ```
 Select timeseries(order_time, 'month'), sum(quantity) from sales group by timeseries(order_time,
   'month')
@@ -155,9 +146,21 @@ the future CarbonData release.
       
 
 ## Compacting timeseries datamp
-Refer to Compaction section in [preaggregation datamap](https://github.com/apache/carbondata/blob/master/docs/datamap/preaggregate-datamap-guide.md). 
+Refer to Compaction section in [preaggregation datamap](./preaggregate-datamap-guide.md). 
 Same applies to timeseries datamap.
 
 ## Data Management on timeseries datamap
-Refer to Data Management section in [preaggregation datamap](https://github.com/apache/carbondata/blob/master/docs/datamap/preaggregate-datamap-guide.md).
-Same applies to timeseries datamap.
\ No newline at end of file
+Refer to Data Management section in [preaggregation datamap](./preaggregate-datamap-guide.md).
+Same applies to timeseries datamap.
+
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/troubleshooting.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/troubleshooting.md b/src/site/markdown/troubleshooting.md
deleted file mode 100644
index 0156121..0000000
--- a/src/site/markdown/troubleshooting.md
+++ /dev/null
@@ -1,267 +0,0 @@
-<!--
-    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 exception
-   ```
-
-  **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 exception
-   ```
-
-  **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.

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/useful-tips-on-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/useful-tips-on-carbondata.md b/src/site/markdown/useful-tips-on-carbondata.md
deleted file mode 100644
index 641a7f3..0000000
--- a/src/site/markdown/useful-tips-on-carbondata.md
+++ /dev/null
@@ -1,179 +0,0 @@
-<!--
-    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.
--->
-
-# Useful Tips
-  This tutorial guides you to create CarbonData Tables and optimize performance.
-  The following sections will elaborate on the below topics :
-
-  * [Suggestions to create CarbonData Table](#suggestions-to-create-carbondata-table)
-  * [Configuration for Optimizing Data Loading performance for Massive Data](#configuration-for-optimizing-data-loading-performance-for-massive-data)
-  * [Optimizing Mass Data Loading](#configurations-for-optimizing-carbondata-performance)
-
-## Suggestions to Create CarbonData Table
-
-  For example, the results of the analysis for table creation with dimensions ranging from 10 thousand to 10 billion rows and 100 to 300 columns have been summarized below.
-  The following table describes some of the columns from the table used.
-
-  - **Table Column Description**
-
-  | Column Name | Data Type     | Cardinality | Attribution |
-  |-------------|---------------|-------------|-------------|
-  | msisdn      | String        | 30 million  | Dimension   |
-  | BEGIN_TIME  | BigInt        | 10 Thousand | Dimension   |
-  | HOST        | String        | 1 million   | Dimension   |
-  | Dime_1      | String        | 1 Thousand  | Dimension   |
-  | counter_1   | Decimal       | NA          | Measure     |
-  | counter_2   | Numeric(20,0) | NA          | Measure     |
-  | ...         | ...           | NA          | Measure     |
-  | counter_100 | Decimal       | NA          | Measure     |
-
-
-  - **Put the frequently-used column filter in the beginning**
-
-  For example, MSISDN filter is used in most of the query then we must put the MSISDN in the first column.
-  The create table command can be modified as suggested below :
-
-  ```
-  create table carbondata_table(
-    msisdn String,
-    BEGIN_TIME bigint,
-    HOST String,
-    Dime_1 String,
-    counter_1, Decimal
-    ...
-    
-    )STORED BY 'carbondata'
-    TBLPROPERTIES ('SORT_COLUMNS'='msisdn, Dime_1')
-  ```
-
-  Now the query with MSISDN in the filter will be more efficient.
-
-  - **Put the frequently-used columns in the order of low to high cardinality**
-
-  If the table in the specified query has multiple columns which are frequently used to filter the results, it is suggested to put
-  the columns in the order of cardinality low to high. This ordering of frequently used columns improves the compression ratio and
-  enhances the performance of queries with filter on these columns.
-
-  For example, if MSISDN, HOST and Dime_1 are frequently-used columns, then the column order of table is suggested as
-  Dime_1>HOST>MSISDN, because Dime_1 has the lowest cardinality.
-  The create table command can be modified as suggested below :
-
-  ```
-  create table carbondata_table(
-      msisdn String,
-      BEGIN_TIME bigint,
-      HOST String,
-      Dime_1 String,
-      counter_1, Decimal
-      ...
-      
-      )STORED BY 'carbondata'
-      TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
-  ```
-
-  - **For measure type columns with non high accuracy, replace Numeric(20,0) data type with Double data type**
-
-  For columns of measure type, not requiring high accuracy, it is suggested to replace Numeric data type with Double to enhance query performance. 
-  The create table command can be modified as below :
-
-```
-  create table carbondata_table(
-    Dime_1 String,
-    BEGIN_TIME bigint,
-    END_TIME bigint,
-    HOST String,
-    MSISDN String,
-    counter_1 decimal,
-    counter_2 double,
-    ...
-    )STORED BY 'carbondata'
-    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
-```
-  The result of performance analysis of test-case shows reduction in query execution time from 15 to 3 seconds, thereby improving performance by nearly 5 times.
-
- - **Columns of incremental character should be re-arranged at the end of dimensions**
-
-  Consider the following scenario where data is loaded each day and the begin_time is incremental for each load, it is suggested to put begin_time at the end of dimensions.
-  Incremental values are efficient in using min/max index. The create table command can be modified as below :
-
-  ```
-  create table carbondata_table(
-    Dime_1 String,
-    HOST String,
-    MSISDN String,
-    counter_1 double,
-    counter_2 double,
-    BEGIN_TIME bigint,
-    END_TIME bigint,
-    ...
-    counter_100 double
-    )STORED BY 'carbondata'
-    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
-  ```
-
-  **NOTE:**
-  + BloomFilter can be created to enhance performance for queries with precise equal/in conditions. You can find more information about it in BloomFilter datamap [document](https://github.com/apache/carbondata/blob/master/docs/datamap/bloomfilter-datamap-guide.md).
-
-
-## Configuration for Optimizing Data Loading performance for Massive Data
-
-
-  CarbonData supports large data load, in this process sorting data while loading consumes a lot of memory and disk IO and
-  this can result sometimes in "Out Of Memory" exception.
-  If you do not have much memory to use, then you may prefer to slow the speed of data loading instead of data load failure.
-  You can configure CarbonData by tuning following properties in carbon.properties file to get a better performance.
-
-  | Parameter | Default Value | Description/Tuning |
-  |-----------|-------------|--------|
-  |carbon.number.of.cores.while.loading|Default: 2.This value should be >= 2|Specifies the number of cores used for data processing during data loading in CarbonData. |
-  |carbon.sort.size|Default: 100000. The value should be >= 100.|Threshold to write local file in sort step when loading data|
-  |carbon.sort.file.write.buffer.size|Default:  50000.|DataOutputStream buffer. |
-  |carbon.number.of.cores.block.sort|Default: 7 | If you have huge memory and CPUs, increase it as you will|
-  |carbon.merge.sort.reader.thread|Default: 3 |Specifies the number of cores used for temp file merging during data loading in CarbonData.|
-  |carbon.merge.sort.prefetch|Default: true | You may want set this value to false if you have not enough memory|
-
-  For example, if there are 10 million records, and i have only 16 cores, 64GB memory, will be loaded to CarbonData table.
-  Using the default configuration  always fail in sort step. Modify carbon.properties as suggested below:
-
-  ```
-  carbon.number.of.cores.block.sort=1
-  carbon.merge.sort.reader.thread=1
-  carbon.sort.size=5000
-  carbon.sort.file.write.buffer.size=5000
-  carbon.merge.sort.prefetch=false
-  ```
-
-## Configurations for Optimizing CarbonData Performance
-
-  Recently we did some performance POC on CarbonData for Finance and telecommunication Field. It involved detailed queries and aggregation
-  scenarios. After the completion of POC, some of the configurations impacting the performance have been identified and tabulated below :
-
-  | Parameter | Location | Used For  | Description | Tuning |
-  |----------------------------------------------|-----------------------------------|---------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-  | carbon.sort.intermediate.files.limit | spark/carbonlib/carbon.properties | Data loading | During the loading of data, local temp is used to sort the data. This number specifies the minimum number of intermediate files after which the  merge sort has to be initiated. | Increasing the parameter to a higher value will improve the load performance. For example, when we increase the value from 20 to 100, it increases the data load performance from 35MB/S to more than 50MB/S. Higher values of this parameter consumes  more memory during the load. |
-  | carbon.number.of.cores.while.loading | spark/carbonlib/carbon.properties | Data loading | Specifies the number of cores used for data processing during data loading in CarbonData. | If you have more number of CPUs, then you can increase the number of CPUs, which will increase the performance. For example if we increase the value from 2 to 4 then the CSV reading performance can increase about 1 times |
-  | carbon.compaction.level.threshold | spark/carbonlib/carbon.properties | Data loading and Querying | For minor compaction, specifies the number of segments to be merged in stage 1 and number of compacted segments to be merged in stage 2. | Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance. Configuring this parameter will merge the small segment to one big segment which will sort the data and improve the performance. For Example in one telecommunication scenario, the performance improves about 2 times after minor compaction. |
-  | spark.sql.shuffle.partitions | spark/conf/spark-defaults.conf | Querying | The number of task started when spark shuffle. | The value can be 1 to 2 times as much as the executor cores. In an aggregation scenario, reducing the number from 200 to 32 reduced the query time from 17 to 9 seconds. |
-  | spark.executor.instances/spark.executor.cores/spark.executor.memory | spark/conf/spark-defaults.conf | Querying | The number of executors, CPU cores, and memory used for CarbonData query. | In the bank scenario, we provide the 4 CPUs cores and 15 GB for each executor which can get good performance. This 2 value does not mean more the better. It needs to be configured properly in case of limited resources. For example, In the bank scenario, it has enough CPU 32 cores each node but less memory 64 GB each node. So we cannot give more CPU but less memory. For example, when 4 cores and 12GB for each executor. It sometimes happens GC during the query which impact the query performance very much from the 3 second to more than 15 seconds. In this scenario need to increase the memory or decrease the CPU cores. |
-  | carbon.detail.batch.size | spark/carbonlib/carbon.properties | Data loading | The buffer size to store records, returned from the block scan. | In limit scenario this parameter is very important. For example your query limit is 1000. But if we set this value to 3000 that means we get 3000 records from scan but spark will only take 1000 rows. So the 2000 remaining are useless. In one Finance test case after we set it to 100, in the limit 1000 scenario the performance increase about 2 times in comparison to if we set this value to 12000. |
-  | carbon.use.local.dir | spark/carbonlib/carbon.properties | Data loading | Whether use YARN local directories for multi-table load disk load balance | If this is set it to true CarbonData will use YARN local directories for multi-table load disk load balance, that will improve the data load performance. |
-  | carbon.use.multiple.temp.dir | spark/carbonlib/carbon.properties | Data loading | Whether to use multiple YARN local directories during table data loading for disk load balance | After enabling 'carbon.use.local.dir', if this is set to true, CarbonData will use all YARN local directories during data load for disk load balance, that will improve the data load performance. Please enable this property when you encounter disk hotspot problem during data loading. |
-  | carbon.sort.temp.compressor | spark/carbonlib/carbon.properties | Data loading | Specify the name of compressor to compress the intermediate sort temporary files during sort procedure in data loading. | The optional values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files. This parameter will be useful if you encounter disk bottleneck. |
-  | carbon.load.skewedDataOptimization.enabled | spark/carbonlib/carbon.properties | Data loading | Whether to enable size based block allocation strategy for data loading. | When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data -- It's useful if the size of your input data files varies widely, say 1MB~1GB. |
-  | carbon.load.min.size.enabled | spark/carbonlib/carbon.properties | Data loading | Whether to enable node minumun input data size allocation strategy for data loading.| When loading, carbondata will use node minumun input data size allocation strategy for task distribution. It will make sure the node load the minimum amount of data -- It's useful if the size of your input data files very small, say 1MB~256MB,Avoid generating a large number of small files. |
-  
-  Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.


[08/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/performance-tuning.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/performance-tuning.html b/src/main/webapp/performance-tuning.html
new file mode 100644
index 0000000..49b3d3a
--- /dev/null
+++ b/src/main/webapp/performance-tuning.html
@@ -0,0 +1,529 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="useful-tips" class="anchor" href="#useful-tips" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Useful Tips</h1>
+<p>This tutorial guides you to create CarbonData Tables and optimize performance.
+The following sections will elaborate on the below topics :</p>
+<ul>
+<li><a href="#suggestions-to-create-carbondata-table">Suggestions to create CarbonData Table</a></li>
+<li><a href="#configuration-for-optimizing-data-loading-performance-for-massive-data">Configuration for Optimizing Data Loading performance for Massive Data</a></li>
+<li><a href="#configurations-for-optimizing-carbondata-performance">Optimizing Query Performance</a></li>
+</ul>
+<h2>
+<a id="suggestions-to-create-carbondata-table" class="anchor" href="#suggestions-to-create-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Suggestions to Create CarbonData Table</h2>
+<p>For example, the results of the analysis for table creation with dimensions ranging from 10 thousand to 10 billion rows and 100 to 300 columns have been summarized below.
+The following table describes some of the columns from the table used.</p>
+<ul>
+<li><strong>Table Column Description</strong></li>
+</ul>
+<table>
+<thead>
+<tr>
+<th>Column Name</th>
+<th>Data Type</th>
+<th>Cardinality</th>
+<th>Attribution</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>msisdn</td>
+<td>String</td>
+<td>30 million</td>
+<td>Dimension</td>
+</tr>
+<tr>
+<td>BEGIN_TIME</td>
+<td>BigInt</td>
+<td>10 Thousand</td>
+<td>Dimension</td>
+</tr>
+<tr>
+<td>HOST</td>
+<td>String</td>
+<td>1 million</td>
+<td>Dimension</td>
+</tr>
+<tr>
+<td>Dime_1</td>
+<td>String</td>
+<td>1 Thousand</td>
+<td>Dimension</td>
+</tr>
+<tr>
+<td>counter_1</td>
+<td>Decimal</td>
+<td>NA</td>
+<td>Measure</td>
+</tr>
+<tr>
+<td>counter_2</td>
+<td>Numeric(20,0)</td>
+<td>NA</td>
+<td>Measure</td>
+</tr>
+<tr>
+<td>...</td>
+<td>...</td>
+<td>NA</td>
+<td>Measure</td>
+</tr>
+<tr>
+<td>counter_100</td>
+<td>Decimal</td>
+<td>NA</td>
+<td>Measure</td>
+</tr>
+</tbody>
+</table>
+<ul>
+<li><strong>Put the frequently-used column filter in the beginning of SORT_COLUMNS</strong></li>
+</ul>
+<p>For example, MSISDN filter is used in most of the query then we must put the MSISDN as the first column in SORT_COLUMNS property.
+The create table command can be modified as suggested below :</p>
+<pre><code>create table carbondata_table(
+  msisdn String,
+  BEGIN_TIME bigint,
+  HOST String,
+  Dime_1 String,
+  counter_1, Decimal
+  ...
+  
+  )STORED BY 'carbondata'
+  TBLPROPERTIES ('SORT_COLUMNS'='msisdn, Dime_1')
+</code></pre>
+<p>Now the query with MSISDN in the filter will be more efficient.</p>
+<ul>
+<li><strong>Put the frequently-used columns in the order of low to high cardinality in SORT_COLUMNS</strong></li>
+</ul>
+<p>If the table in the specified query has multiple columns which are frequently used to filter the results, it is suggested to put
+the columns in the order of cardinality low to high in SORT_COLUMNS configuration. This ordering of frequently used columns improves the compression ratio and
+enhances the performance of queries with filter on these columns.</p>
+<p>For example, if MSISDN, HOST and Dime_1 are frequently-used columns, then the column order of table is suggested as
+Dime_1&gt;HOST&gt;MSISDN, because Dime_1 has the lowest cardinality.
+The create table command can be modified as suggested below :</p>
+<pre><code>create table carbondata_table(
+    msisdn String,
+    BEGIN_TIME bigint,
+    HOST String,
+    Dime_1 String,
+    counter_1, Decimal
+    ...
+    
+    )STORED BY 'carbondata'
+    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
+</code></pre>
+<ul>
+<li><strong>For measure type columns with non high accuracy, replace Numeric(20,0) data type with Double data type</strong></li>
+</ul>
+<p>For columns of measure type, not requiring high accuracy, it is suggested to replace Numeric data type with Double to enhance query performance.
+The create table command can be modified as below :</p>
+<pre><code>  create table carbondata_table(
+    Dime_1 String,
+    BEGIN_TIME bigint,
+    END_TIME bigint,
+    HOST String,
+    MSISDN String,
+    counter_1 decimal,
+    counter_2 double,
+    ...
+    )STORED BY 'carbondata'
+    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
+</code></pre>
+<p>The result of performance analysis of test-case shows reduction in query execution time from 15 to 3 seconds, thereby improving performance by nearly 5 times.</p>
+<ul>
+<li><strong>Columns of incremental character should be re-arranged at the end of dimensions</strong></li>
+</ul>
+<p>Consider the following scenario where data is loaded each day and the begin_time is incremental for each load, it is suggested to put begin_time at the end of dimensions.
+Incremental values are efficient in using min/max index. The create table command can be modified as below :</p>
+<pre><code>create table carbondata_table(
+  Dime_1 String,
+  HOST String,
+  MSISDN String,
+  counter_1 double,
+  counter_2 double,
+  BEGIN_TIME bigint,
+  END_TIME bigint,
+  ...
+  counter_100 double
+  )STORED BY 'carbondata'
+  TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>BloomFilter can be created to enhance performance for queries with precise equal/in conditions. You can find more information about it in BloomFilter datamap <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/bloomfilter-datamap-guide.html" target=_blank>document</a>.</li>
+</ul>
+<h2>
+<a id="configuration-for-optimizing-data-loading-performance-for-massive-data" class="anchor" href="#configuration-for-optimizing-data-loading-performance-for-massive-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configuration for Optimizing Data Loading performance for Massive Data</h2>
+<p>CarbonData supports large data load, in this process sorting data while loading consumes a lot of memory and disk IO and
+this can result sometimes in "Out Of Memory" exception.
+If you do not have much memory to use, then you may prefer to slow the speed of data loading instead of data load failure.
+You can configure CarbonData by tuning following properties in carbon.properties file to get a better performance.</p>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Default Value</th>
+<th>Description/Tuning</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>carbon.number.of.cores.while.loading</td>
+<td>Default: 2.This value should be &gt;= 2</td>
+<td>Specifies the number of cores used for data processing during data loading in CarbonData.</td>
+</tr>
+<tr>
+<td>carbon.sort.size</td>
+<td>Default: 100000. The value should be &gt;= 100.</td>
+<td>Threshold to write local file in sort step when loading data</td>
+</tr>
+<tr>
+<td>carbon.sort.file.write.buffer.size</td>
+<td>Default:  50000.</td>
+<td>DataOutputStream buffer.</td>
+</tr>
+<tr>
+<td>carbon.merge.sort.reader.thread</td>
+<td>Default: 3</td>
+<td>Specifies the number of cores used for temp file merging during data loading in CarbonData.</td>
+</tr>
+<tr>
+<td>carbon.merge.sort.prefetch</td>
+<td>Default: true</td>
+<td>You may want set this value to false if you have not enough memory</td>
+</tr>
+</tbody>
+</table>
+<p>For example, if there are 10 million records, and i have only 16 cores, 64GB memory, will be loaded to CarbonData table.
+Using the default configuration  always fail in sort step. Modify carbon.properties as suggested below:</p>
+<pre><code>carbon.merge.sort.reader.thread=1
+carbon.sort.size=5000
+carbon.sort.file.write.buffer.size=5000
+carbon.merge.sort.prefetch=false
+</code></pre>
+<h2>
+<a id="configurations-for-optimizing-carbondata-performance" class="anchor" href="#configurations-for-optimizing-carbondata-performance" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configurations for Optimizing CarbonData Performance</h2>
+<p>Recently we did some performance POC on CarbonData for Finance and telecommunication Field. It involved detailed queries and aggregation
+scenarios. After the completion of POC, some of the configurations impacting the performance have been identified and tabulated below :</p>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Location</th>
+<th>Used For</th>
+<th>Description</th>
+<th>Tuning</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>carbon.sort.intermediate.files.limit</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>During the loading of data, local temp is used to sort the data. This number specifies the minimum number of intermediate files after which the  merge sort has to be initiated.</td>
+<td>Increasing the parameter to a higher value will improve the load performance. For example, when we increase the value from 20 to 100, it increases the data load performance from 35MB/S to more than 50MB/S. Higher values of this parameter consumes  more memory during the load.</td>
+</tr>
+<tr>
+<td>carbon.number.of.cores.while.loading</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Specifies the number of cores used for data processing during data loading in CarbonData.</td>
+<td>If you have more number of CPUs, then you can increase the number of CPUs, which will increase the performance. For example if we increase the value from 2 to 4 then the CSV reading performance can increase about 1 times</td>
+</tr>
+<tr>
+<td>carbon.compaction.level.threshold</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading and Querying</td>
+<td>For minor compaction, specifies the number of segments to be merged in stage 1 and number of compacted segments to be merged in stage 2.</td>
+<td>Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance. Configuring this parameter will merge the small segment to one big segment which will sort the data and improve the performance. For Example in one telecommunication scenario, the performance improves about 2 times after minor compaction.</td>
+</tr>
+<tr>
+<td>spark.sql.shuffle.partitions</td>
+<td>spark/conf/spark-defaults.conf</td>
+<td>Querying</td>
+<td>The number of task started when spark shuffle.</td>
+<td>The value can be 1 to 2 times as much as the executor cores. In an aggregation scenario, reducing the number from 200 to 32 reduced the query time from 17 to 9 seconds.</td>
+</tr>
+<tr>
+<td>spark.executor.instances/spark.executor.cores/spark.executor.memory</td>
+<td>spark/conf/spark-defaults.conf</td>
+<td>Querying</td>
+<td>The number of executors, CPU cores, and memory used for CarbonData query.</td>
+<td>In the bank scenario, we provide the 4 CPUs cores and 15 GB for each executor which can get good performance. This 2 value does not mean more the better. It needs to be configured properly in case of limited resources. For example, In the bank scenario, it has enough CPU 32 cores each node but less memory 64 GB each node. So we cannot give more CPU but less memory. For example, when 4 cores and 12GB for each executor. It sometimes happens GC during the query which impact the query performance very much from the 3 second to more than 15 seconds. In this scenario need to increase the memory or decrease the CPU cores.</td>
+</tr>
+<tr>
+<td>carbon.detail.batch.size</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>The buffer size to store records, returned from the block scan.</td>
+<td>In limit scenario this parameter is very important. For example your query limit is 1000. But if we set this value to 3000 that means we get 3000 records from scan but spark will only take 1000 rows. So the 2000 remaining are useless. In one Finance test case after we set it to 100, in the limit 1000 scenario the performance increase about 2 times in comparison to if we set this value to 12000.</td>
+</tr>
+<tr>
+<td>carbon.use.local.dir</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Whether use YARN local directories for multi-table load disk load balance</td>
+<td>If this is set it to true CarbonData will use YARN local directories for multi-table load disk load balance, that will improve the data load performance.</td>
+</tr>
+<tr>
+<td>carbon.use.multiple.temp.dir</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Whether to use multiple YARN local directories during table data loading for disk load balance</td>
+<td>After enabling 'carbon.use.local.dir', if this is set to true, CarbonData will use all YARN local directories during data load for disk load balance, that will improve the data load performance. Please enable this property when you encounter disk hotspot problem during data loading.</td>
+</tr>
+<tr>
+<td>carbon.sort.temp.compressor</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Specify the name of compressor to compress the intermediate sort temporary files during sort procedure in data loading.</td>
+<td>The optional values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files. This parameter will be useful if you encounter disk bottleneck.</td>
+</tr>
+<tr>
+<td>carbon.load.skewedDataOptimization.enabled</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Whether to enable size based block allocation strategy for data loading.</td>
+<td>When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data -- It's useful if the size of your input data files varies widely, say 1MB~1GB.</td>
+</tr>
+<tr>
+<td>carbon.load.min.size.enabled</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Whether to enable node minumun input data size allocation strategy for data loading.</td>
+<td>When loading, carbondata will use node minumun input data size allocation strategy for task distribution. It will make sure the node load the minimum amount of data -- It's useful if the size of your input data files very small, say 1MB~256MB,Avoid generating a large number of small files.</td>
+</tr>
+</tbody>
+</table>
+<p>Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.</p>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__perf').addClass('selected'); });
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/preaggregate-datamap-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/preaggregate-datamap-guide.html b/src/main/webapp/preaggregate-datamap-guide.html
index d68764d..9220c84 100644
--- a/src/main/webapp/preaggregate-datamap-guide.html
+++ b/src/main/webapp/preaggregate-datamap-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -266,7 +317,7 @@ kinds of DataMap:</p>
 a. 'path' is used to specify the store location of the datamap.('path'='/location/').
 b. 'partitioning' when set to false enables user to disable partitioning of the datamap.
 Default value is true for this property.</li>
-<li>timeseries, for timeseries roll-up table. Please refer to <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/timeseries-datamap-guide.html" target=_blank>Timeseries DataMap</a>
+<li>timeseries, for timeseries roll-up table. Please refer to <a href="./timeseries-datamap-guide.html">Timeseries DataMap</a>
 </li>
 </ol>
 <p>DataMap can be dropped using following DDL</p>
@@ -415,6 +466,17 @@ release, user can do as following:</p>
 <li>Create the pre-aggregate table again by <code>CREATE DATAMAP</code> command
 Basically, user can manually trigger the operation by re-building the datamap.</li>
 </ol>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -430,4 +492,4 @@ Basically, user can manually trigger the operation by re-building the datamap.</
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/quick-start-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/quick-start-guide.html b/src/main/webapp/quick-start-guide.html
index 89380b4..ea88086 100644
--- a/src/main/webapp/quick-start-guide.html
+++ b/src/main/webapp/quick-start-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -161,12 +212,12 @@
                                     <div>
 <h1>
 <a id="quick-start" class="anchor" href="#quick-start" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Quick Start</h1>
-<p>This tutorial provides a quick introduction to using CarbonData.</p>
+<p>This tutorial provides a quick introduction to using CarbonData.To follow along with this guide, first download a packaged release of CarbonData from the <a href="https://dist.apache.org/repos/dist/release/carbondata/" target=_blank rel="nofollow">CarbonData website</a>.Alternatively it can be created following <a href="https://github.com/apache/carbondata/tree/master/build" target=_blank>Building CarbonData</a> steps.</p>
 <h2>
 <a id="prerequisites" class="anchor" href="#prerequisites" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h2>
 <ul>
 <li>
-<p><a href="https://github.com/apache/carbondata/blob/master/build" target=_blank>Installation and building CarbonData</a>.</p>
+<p>Spark 2.2.1 version is installed and running.CarbonData supports Spark versions upto 2.2.1.Please follow steps described in <a href="https://spark.apache.org/docs/latest" target=_blank rel="nofollow">Spark docs website</a> for installing and running Spark.</p>
 </li>
 <li>
 <p>Create a sample.csv file using the following commands. The CSV file is required for loading data into CarbonData.</p>
@@ -181,14 +232,29 @@ EOF
 </li>
 </ul>
 <h2>
-<a id="interactive-analysis-with-spark-shell-version-21" class="anchor" href="#interactive-analysis-with-spark-shell-version-21" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Interactive Analysis with Spark Shell Version 2.1</h2>
+<a id="deployment-modes" class="anchor" href="#deployment-modes" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deployment modes</h2>
+<p>CarbonData can be integrated with Spark and Presto Execution Engines.The below documentation guides on Installing and Configuring with these execution engines.</p>
+<h3>
+<a id="spark" class="anchor" href="#spark" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Spark</h3>
+<p><a href="#installing-and-configuring-carbondata-to-run-locally-with-spark-shell">Installing and Configuring CarbonData to run locally with Spark Shell</a></p>
+<p><a href="#installing-and-configuring-carbondata-on-standalone-spark-cluster">Installing and Configuring CarbonData on Standalone Spark Cluster</a></p>
+<p><a href="#installing-and-configuring-carbondata-on-spark-on-yarn-cluster">Installing and Configuring CarbonData on Spark on YARN Cluster</a></p>
+<h3>
+<a id="presto" class="anchor" href="#presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Presto</h3>
+<p><a href="#installing-and-configuring-carbondata-on-presto">Installing and Configuring CarbonData on Presto</a></p>
+<h2>
+<a id="querying-data" class="anchor" href="#querying-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Querying Data</h2>
+<p><a href="#query-execution-using-carbondata-thrift-server">Query Execution using CarbonData Thrift Server</a></p>
+<h2></h2>
+<h2>
+<a id="installing-and-configuring-carbondata-to-run-locally-with-spark-shell" class="anchor" href="#installing-and-configuring-carbondata-to-run-locally-with-spark-shell" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData to run locally with Spark Shell</h2>
 <p>Apache Spark Shell provides a simple way to learn the API, as well as a powerful tool to analyze data interactively. Please visit <a href="http://spark.apache.org/docs/latest/" target=_blank rel="nofollow">Apache Spark Documentation</a> for more details on Spark shell.</p>
 <h4>
 <a id="basics" class="anchor" href="#basics" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Basics</h4>
 <p>Start Spark shell by running the following command in the Spark directory:</p>
 <pre><code>./bin/spark-shell --jars &lt;carbondata assembly jar path&gt;
 </code></pre>
-<p><strong>NOTE</strong>: Assembly jar will be available after <a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>building CarbonData</a> and can be copied from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code></p>
+<p><strong>NOTE</strong>: Path where packaged release of CarbonData was downloaded or assembly jar will be available after <a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>building CarbonData</a> and can be copied from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code></p>
 <p>In this shell, SparkSession is readily available as <code>spark</code> and Spark context is readily available as <code>sc</code>.</p>
 <p>In order to create a CarbonSession we will have to configure it explicitly in the following manner :</p>
 <ul>
@@ -203,7 +269,7 @@ import org.apache.spark.sql.CarbonSession._
 <pre><code>val carbon = SparkSession.builder().config(sc.getConf)
              .getOrCreateCarbonSession("&lt;hdfs store path&gt;")
 </code></pre>
-<p><strong>NOTE</strong>: By default metastore location is pointed to <code>../carbon.metastore</code>, user can provide own metastore location to CarbonSession like <code>SparkSession.builder().config(sc.getConf) .getOrCreateCarbonSession("&lt;hdfs store path&gt;", "&lt;local metastore path&gt;")</code></p>
+<p><strong>NOTE</strong>: By default metastore location points to <code>../carbon.metastore</code>, user can provide own metastore location to CarbonSession like <code>SparkSession.builder().config(sc.getConf) .getOrCreateCarbonSession("&lt;hdfs store path&gt;", "&lt;local metastore path&gt;")</code></p>
 <h4>
 <a id="executing-queries" class="anchor" href="#executing-queries" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Executing Queries</h4>
 <h6>
@@ -222,7 +288,7 @@ import org.apache.spark.sql.CarbonSession._
                   INTO TABLE test_table")
 </code></pre>
 <p><strong>NOTE</strong>: Please provide the real file path of <code>sample.csv</code> for the above script.
-If you get "tablestatus.lock" issue, please refer to <a href="troubleshooting.html">troubleshooting</a></p>
+If you get "tablestatus.lock" issue, please refer to <a href="faq.html">FAQ</a></p>
 <h6>
 <a id="query-data-from-a-table" class="anchor" href="#query-data-from-a-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Query Data from a Table</h6>
 <pre><code>scala&gt;carbon.sql("SELECT * FROM test_table").show()
@@ -231,6 +297,398 @@ scala&gt;carbon.sql("SELECT city, avg(age), sum(age)
                   FROM test_table
                   GROUP BY city").show()
 </code></pre>
+<h2>
+<a id="installing-and-configuring-carbondata-on-standalone-spark-cluster" class="anchor" href="#installing-and-configuring-carbondata-on-standalone-spark-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Standalone Spark Cluster</h2>
+<h3>
+<a id="prerequisites-1" class="anchor" href="#prerequisites-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h3>
+<ul>
+<li>Hadoop HDFS and Yarn should be installed and running.</li>
+<li>Spark should be installed and running on all the cluster nodes.</li>
+<li>CarbonData user should have permission to access HDFS.</li>
+</ul>
+<h3>
+<a id="procedure" class="anchor" href="#procedure" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Procedure</h3>
+<ol>
+<li>
+<p><a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>Build the CarbonData</a> project and get the assembly jar from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code>.</p>
+</li>
+<li>
+<p>Copy <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code> to <code>$SPARK_HOME/carbonlib</code> folder.</p>
+<p><strong>NOTE</strong>: Create the carbonlib folder if it does not exist inside <code>$SPARK_HOME</code> path.</p>
+</li>
+<li>
+<p>Add the carbonlib folder path in the Spark classpath. (Edit <code>$SPARK_HOME/conf/spark-env.sh</code> file and modify the value of <code>SPARK_CLASSPATH</code> by appending <code>$SPARK_HOME/carbonlib/*</code> to the existing value)</p>
+</li>
+<li>
+<p>Copy the <code>./conf/carbon.properties.template</code> file from CarbonData repository to <code>$SPARK_HOME/conf/</code> folder and rename the file to <code>carbon.properties</code>.</p>
+</li>
+<li>
+<p>Repeat Step 2 to Step 5 in all the nodes of the cluster.</p>
+</li>
+<li>
+<p>In Spark node[master], configure the properties mentioned in the following table in <code>$SPARK_HOME/conf/spark-defaults.conf</code> file.</p>
+</li>
+</ol>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Value</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>spark.driver.extraJavaOptions</td>
+<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
+<td>A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.</td>
+</tr>
+<tr>
+<td>spark.executor.extraJavaOptions</td>
+<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
+<td>A string of extra JVM options to pass to executors. For instance, GC settings or other logging. <strong>NOTE</strong>: You can enter multiple values separated by space.</td>
+</tr>
+</tbody>
+</table>
+<ol>
+<li>Add the following properties in <code>$SPARK_HOME/conf/carbon.properties</code> file:</li>
+</ol>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Required</th>
+<th>Description</th>
+<th>Example</th>
+<th>Remark</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>carbon.storelocation</td>
+<td>NO</td>
+<td>Location where data CarbonData will create the store and write the data in its own format. If not specified then it takes spark.sql.warehouse.dir path.</td>
+<td>hdfs://HOSTNAME:PORT/Opt/CarbonStore</td>
+<td>Propose to set HDFS directory</td>
+</tr>
+</tbody>
+</table>
+<ol>
+<li>Verify the installation. For example:</li>
+</ol>
+<pre><code>./spark-shell --master spark://HOSTNAME:PORT --total-executor-cores 2
+--executor-memory 2G
+</code></pre>
+<p><strong>NOTE</strong>: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.</p>
+<h2>
+<a id="installing-and-configuring-carbondata-on-spark-on-yarn-cluster" class="anchor" href="#installing-and-configuring-carbondata-on-spark-on-yarn-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Spark on YARN Cluster</h2>
+<p>This section provides the procedure to install CarbonData on "Spark on YARN" cluster.</p>
+<h3>
+<a id="prerequisites-2" class="anchor" href="#prerequisites-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h3>
+<ul>
+<li>Hadoop HDFS and Yarn should be installed and running.</li>
+<li>Spark should be installed and running in all the clients.</li>
+<li>CarbonData user should have permission to access HDFS.</li>
+</ul>
+<h3>
+<a id="procedure-1" class="anchor" href="#procedure-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Procedure</h3>
+<p>The following steps are only for Driver Nodes. (Driver nodes are the one which starts the spark context.)</p>
+<ol>
+<li>
+<p><a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>Build the CarbonData</a> project and get the assembly jar from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code> and copy to <code>$SPARK_HOME/carbonlib</code> folder.</p>
+<p><strong>NOTE</strong>: Create the carbonlib folder if it does not exists inside <code>$SPARK_HOME</code> path.</p>
+</li>
+<li>
+<p>Copy the <code>./conf/carbon.properties.template</code> file from CarbonData repository to <code>$SPARK_HOME/conf/</code> folder and rename the file to <code>carbon.properties</code>.</p>
+</li>
+<li>
+<p>Create <code>tar.gz</code> file of carbonlib folder and move it inside the carbonlib folder.</p>
+</li>
+</ol>
+<pre><code>cd $SPARK_HOME
+tar -zcvf carbondata.tar.gz carbonlib/
+mv carbondata.tar.gz carbonlib/
+</code></pre>
+<ol>
+<li>Configure the properties mentioned in the following table in <code>$SPARK_HOME/conf/spark-defaults.conf</code> file.</li>
+</ol>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Description</th>
+<th>Value</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>spark.master</td>
+<td>Set this value to run the Spark in yarn cluster mode.</td>
+<td>Set yarn-client to run the Spark in yarn cluster mode.</td>
+</tr>
+<tr>
+<td>spark.yarn.dist.files</td>
+<td>Comma-separated list of files to be placed in the working directory of each executor.</td>
+<td><code>$SPARK_HOME/conf/carbon.properties</code></td>
+</tr>
+<tr>
+<td>spark.yarn.dist.archives</td>
+<td>Comma-separated list of archives to be extracted into the working directory of each executor.</td>
+<td><code>$SPARK_HOME/carbonlib/carbondata.tar.gz</code></td>
+</tr>
+<tr>
+<td>spark.executor.extraJavaOptions</td>
+<td>A string of extra JVM options to pass to executors. For instance  <strong>NOTE</strong>: You can enter multiple values separated by space.</td>
+<td><code>-Dcarbon.properties.filepath = carbon.properties</code></td>
+</tr>
+<tr>
+<td>spark.executor.extraClassPath</td>
+<td>Extra classpath entries to prepend to the classpath of executors. <strong>NOTE</strong>: If SPARK_CLASSPATH is defined in spark-env.sh, then comment it and append the values in below parameter spark.driver.extraClassPath</td>
+<td><code>carbondata.tar.gz/carbonlib/*</code></td>
+</tr>
+<tr>
+<td>spark.driver.extraClassPath</td>
+<td>Extra classpath entries to prepend to the classpath of the driver. <strong>NOTE</strong>: If SPARK_CLASSPATH is defined in spark-env.sh, then comment it and append the value in below parameter spark.driver.extraClassPath.</td>
+<td><code>$SPARK_HOME/carbonlib/*</code></td>
+</tr>
+<tr>
+<td>spark.driver.extraJavaOptions</td>
+<td>A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.</td>
+<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
+</tr>
+</tbody>
+</table>
+<ol>
+<li>Add the following properties in <code>$SPARK_HOME/conf/carbon.properties</code>:</li>
+</ol>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Required</th>
+<th>Description</th>
+<th>Example</th>
+<th>Default Value</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>carbon.storelocation</td>
+<td>NO</td>
+<td>Location where CarbonData will create the store and write the data in its own format. If not specified then it takes spark.sql.warehouse.dir path.</td>
+<td>hdfs://HOSTNAME:PORT/Opt/CarbonStore</td>
+<td>Propose to set HDFS directory</td>
+</tr>
+</tbody>
+</table>
+<ol>
+<li>Verify the installation.</li>
+</ol>
+<pre><code> ./bin/spark-shell --master yarn-client --driver-memory 1g
+ --executor-cores 2 --executor-memory 2G
+</code></pre>
+<p><strong>NOTE</strong>: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.</p>
+<h2>
+<a id="query-execution-using-carbondata-thrift-server" class="anchor" href="#query-execution-using-carbondata-thrift-server" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Query Execution Using CarbonData Thrift Server</h2>
+<h3>
+<a id="starting-carbondata-thrift-server" class="anchor" href="#starting-carbondata-thrift-server" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Starting CarbonData Thrift Server.</h3>
+<p>a. cd <code>$SPARK_HOME</code></p>
+<p>b. Run the following command to start the CarbonData thrift server.</p>
+<pre><code>./bin/spark-submit
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer
+$SPARK_HOME/carbonlib/$CARBON_ASSEMBLY_JAR &lt;carbon_store_path&gt;
+</code></pre>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Example</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>CARBON_ASSEMBLY_JAR</td>
+<td>CarbonData assembly jar name present in the <code>$SPARK_HOME/carbonlib/</code> folder.</td>
+<td>carbondata_2.xx-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar</td>
+</tr>
+<tr>
+<td>carbon_store_path</td>
+<td>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. If not specified then it takes spark.sql.warehouse.dir path.</td>
+<td><code>hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store</code></td>
+</tr>
+</tbody>
+</table>
+<p><strong>NOTE</strong>: From Spark 1.6, by default the Thrift server runs in multi-session mode. Which means each JDBC/ODBC connection owns a copy of their own SQL configuration and temporary function registry. Cached tables are still shared though. If you prefer to run the Thrift server in single-session mode and share all SQL configuration and temporary function registry, please set option <code>spark.sql.hive.thriftServer.singleSession</code> to <code>true</code>. You may either add this option to <code>spark-defaults.conf</code>, or pass it to <code>spark-submit.sh</code> via <code>--conf</code>:</p>
+<pre><code>./bin/spark-submit
+--conf spark.sql.hive.thriftServer.singleSession=true
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer
+$SPARK_HOME/carbonlib/$CARBON_ASSEMBLY_JAR &lt;carbon_store_path&gt;
+</code></pre>
+<p><strong>But</strong> in single-session mode, if one user changes the database from one connection, the database of the other connections will be changed too.</p>
+<p><strong>Examples</strong></p>
+<ul>
+<li>Start with default memory and executors.</li>
+</ul>
+<pre><code>./bin/spark-submit
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer 
+$SPARK_HOME/carbonlib
+/carbondata_2.xx-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar
+hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
+</code></pre>
+<ul>
+<li>Start with Fixed executors and resources.</li>
+</ul>
+<pre><code>./bin/spark-submit
+--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://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
+</code></pre>
+<h3>
+<a id="connecting-to-carbondata-thrift-server-using-beeline" class="anchor" href="#connecting-to-carbondata-thrift-server-using-beeline" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Connecting to CarbonData Thrift Server Using Beeline.</h3>
+<pre><code>     cd $SPARK_HOME
+     ./sbin/start-thriftserver.sh
+     ./bin/beeline -u jdbc:hive2://&lt;thriftserver_host&gt;:port
+
+     Example
+     ./bin/beeline -u jdbc:hive2://10.10.10.10:10000
+</code></pre>
+<h2>
+<a id="installing-and-configuring-carbondata-on-presto" class="anchor" href="#installing-and-configuring-carbondata-on-presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Presto</h2>
+<ul>
+<li>
+<h3>
+<a id="installing-presto" class="anchor" href="#installing-presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing Presto</h3>
+</li>
+</ul>
+<ol>
+<li>
+<p>Download the 0.187 version of Presto using:
+<code>wget https://repo1.maven.org/maven2/com/facebook/presto/presto-server/0.187/presto-server-0.187.tar.gz</code></p>
+</li>
+<li>
+<p>Extract Presto tar file: <code>tar zxvf presto-server-0.187.tar.gz</code>.</p>
+</li>
+<li>
+<p>Download the Presto CLI for the coordinator and name it presto.</p>
+</li>
+</ol>
+<pre><code>  wget https://repo1.maven.org/maven2/com/facebook/presto/presto-cli/0.187/presto-cli-0.187-executable.jar
+
+  mv presto-cli-0.187-executable.jar presto
+
+  chmod +x presto
+</code></pre>
+<h3>
+<a id="create-configuration-files" class="anchor" href="#create-configuration-files" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Configuration Files</h3>
+<ol>
+<li>
+<p>Create <code>etc</code> folder in presto-server-0.187 directory.</p>
+</li>
+<li>
+<p>Create <code>config.properties</code>, <code>jvm.config</code>, <code>log.properties</code>, and <code>node.properties</code> files.</p>
+</li>
+<li>
+<p>Install uuid to generate a node.id.</p>
+<pre><code>sudo apt-get install uuid
+
+uuid
+</code></pre>
+</li>
+</ol>
+<h5>
+<a id="contents-of-your-nodeproperties-file" class="anchor" href="#contents-of-your-nodeproperties-file" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contents of your node.properties file</h5>
+<pre><code>node.environment=production
+node.id=&lt;generated uuid&gt;
+node.data-dir=/home/ubuntu/data
+</code></pre>
+<h5>
+<a id="contents-of-your-jvmconfig-file" class="anchor" href="#contents-of-your-jvmconfig-file" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contents of your jvm.config file</h5>
+<pre><code>-server
+-Xmx16G
+-XX:+UseG1GC
+-XX:G1HeapRegionSize=32M
+-XX:+UseGCOverheadLimit
+-XX:+ExplicitGCInvokesConcurrent
+-XX:+HeapDumpOnOutOfMemoryError
+-XX:OnOutOfMemoryError=kill -9 %p
+</code></pre>
+<h5>
+<a id="contents-of-your-logproperties-file" class="anchor" href="#contents-of-your-logproperties-file" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contents of your log.properties file</h5>
+<pre><code>com.facebook.presto=INFO
+</code></pre>
+<p>The default minimum level is <code>INFO</code>. There are four levels: <code>DEBUG</code>, <code>INFO</code>, <code>WARN</code> and <code>ERROR</code>.</p>
+<h3>
+<a id="coordinator-configurations" class="anchor" href="#coordinator-configurations" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Coordinator Configurations</h3>
+<h5>
+<a id="contents-of-your-configproperties" class="anchor" href="#contents-of-your-configproperties" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contents of your config.properties</h5>
+<pre><code>coordinator=true
+node-scheduler.include-coordinator=false
+http-server.http.port=8086
+query.max-memory=50GB
+query.max-memory-per-node=2GB
+discovery-server.enabled=true
+discovery.uri=&lt;coordinator_ip&gt;:8086
+</code></pre>
+<p>The options <code>node-scheduler.include-coordinator=false</code> and <code>coordinator=true</code> indicate that the node is the coordinator and tells the coordinator not to do any of the computation work itself and to use the workers.</p>
+<p><strong>Note</strong>: It is recommended to set <code>query.max-memory-per-node</code> to half of the JVM config max memory, though the workload is highly concurrent, lower value for <code>query.max-memory-per-node</code> is to be used.</p>
+<p>Also relation between below two configuration-properties should be like:
+If, <code>query.max-memory-per-node=30GB</code>
+Then, <code>query.max-memory=&lt;30GB * number of nodes&gt;</code>.</p>
+<h3>
+<a id="worker-configurations" class="anchor" href="#worker-configurations" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Worker Configurations</h3>
+<h5>
+<a id="contents-of-your-configproperties-1" class="anchor" href="#contents-of-your-configproperties-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contents of your config.properties</h5>
+<pre><code>coordinator=false
+http-server.http.port=8086
+query.max-memory=50GB
+query.max-memory-per-node=2GB
+discovery.uri=&lt;coordinator_ip&gt;:8086
+</code></pre>
+<p><strong>Note</strong>: <code>jvm.config</code> and <code>node.properties</code> files are same for all the nodes (worker + coordinator). All the nodes should have different <code>node.id</code>.(generated by uuid command).</p>
+<h3>
+<a id="catalog-configurations" class="anchor" href="#catalog-configurations" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Catalog Configurations</h3>
+<ol>
+<li>Create a folder named <code>catalog</code> in etc directory of presto on all the nodes of the cluster including the coordinator.</li>
+</ol>
+<h5>
+<a id="configuring-carbondata-in-presto" class="anchor" href="#configuring-carbondata-in-presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configuring Carbondata in Presto</h5>
+<ol>
+<li>Create a file named <code>carbondata.properties</code> in the <code>catalog</code> folder and set the required properties on all the nodes.</li>
+</ol>
+<h3>
+<a id="add-plugins" class="anchor" href="#add-plugins" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Add Plugins</h3>
+<ol>
+<li>Create a directory named <code>carbondata</code> in plugin directory of presto.</li>
+<li>Copy <code>carbondata</code> jars to <code>plugin/carbondata</code> directory on all nodes.</li>
+</ol>
+<h3>
+<a id="start-presto-server-on-all-nodes" class="anchor" href="#start-presto-server-on-all-nodes" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Start Presto Server on all nodes</h3>
+<pre><code>./presto-server-0.187/bin/launcher start
+</code></pre>
+<p>To run it as a background process.</p>
+<pre><code>./presto-server-0.187/bin/launcher run
+</code></pre>
+<p>To run it in foreground.</p>
+<h3>
+<a id="start-presto-cli" class="anchor" href="#start-presto-cli" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Start Presto CLI</h3>
+<pre><code>./presto
+</code></pre>
+<p>To connect to carbondata catalog use the following command:</p>
+<pre><code>./presto --server &lt;coordinator_ip&gt;:8086 --catalog carbondata --schema &lt;schema_name&gt;
+</code></pre>
+<p>Execute the following command to ensure the workers are connected.</p>
+<pre><code>select * from system.runtime.nodes;
+</code></pre>
+<p>Now you can use the Presto CLI on the coordinator to query data sources in the catalog using the Presto workers.</p>
+<p><strong>Note :</strong> Create Tables and data loads should be done before executing queries as we can not create carbon table from this interface.</p>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__quickstart').addClass('selected'); });
+</script>
 </div>
 </div>
 </div>
@@ -246,4 +704,4 @@ scala&gt;carbon.sql("SELECT city, avg(age), sum(age)
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>


[28/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/troubleshooting.html
----------------------------------------------------------------------
diff --git a/content/troubleshooting.html b/content/troubleshooting.html
deleted file mode 100644
index 7e0cd60..0000000
--- a/content/troubleshooting.html
+++ /dev/null
@@ -1,423 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
-                                   target="_blank">Apache CarbonData 1.4.1</a></li>
-							<li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
-                                   target="_blank">Apache CarbonData 1.4.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
-                                   target="_blank">Apache CarbonData 1.3.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
-                                   target="_blank">Apache CarbonData 1.3.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="documentation.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search" target="_blank">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input"  placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="verticalnavbar">
-                <nav class="b-sticky-nav">
-                    <div class="nav-scroller">
-                        <div class="nav__inner">
-                            <a class="b-nav__intro nav__item" href="./videogallery.html">introduction</a>
-                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
-
-                            <div class="nav__item nav__item__with__subs">
-                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
-                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
-                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
-                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
-                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
-                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
-                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
-                            </div>
-
-                            <div class="nav__item nav__item__with__subs">
-                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
-                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
-                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
-                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
-                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
-                            </div>
-
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
-                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
-                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
-                            <a class="b-nav__trouble nav__item" href="./troubleshooting.html">Troubleshooting</a>
-                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
-                            <a class="b-nav__contri nav__item" href="./videogallery.html">Contribute</a>
-                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
-                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
-                        </div>
-                    </div>
-                    <div class="navindicator">
-                        <div class="b-nav__intro navindicator__item"></div>
-                        <div class="b-nav__quickstart navindicator__item"></div>
-                        <div class="b-nav__uses navindicator__item"></div>
-                        <div class="b-nav__docs navindicator__item"></div>
-                        <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
-                        <div class="b-nav__api navindicator__item"></div>
-                        <div class="b-nav__perf navindicator__item"></div>
-                        <div class="b-nav__trouble navindicator__item"></div>
-                        <div class="b-nav__faq navindicator__item"></div>
-                        <div class="b-nav__contri navindicator__item"></div>
-                        <div class="b-nav__security navindicator__item"></div>
-                    </div>
-                </nav>
-            </div>
-            <div class="mdcontent">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="row">
-                                <div class="col-sm-12  col-md-12">
-                                    <div>
-<h1>
-<a id="troubleshooting" class="anchor" href="#troubleshooting" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Troubleshooting</h1>
-<p>This tutorial is designed to provide troubleshooting for end users and developers
-who are building, deploying, and using CarbonData.</p>
-<h2>
-<a id="when-loading-data-gets-tablestatuslock-issues" class="anchor" href="#when-loading-data-gets-tablestatuslock-issues" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>When loading data, gets tablestatus.lock issues:</h2>
-<p><strong>Symptom</strong></p>
-<pre><code>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.&lt;init&gt;(FileOutputStream.java:213)
-	at java.io.FileOutputStream.&lt;init&gt;(FileOutputStream.java:101)
-</code></pre>
-<p><strong>Possible Cause</strong>
-If you use <code>&lt;hdfs path&gt;</code> as store path when creating carbonsession, may get the errors,because the default is LOCALLOCK.</p>
-<p><strong>Procedure</strong>
-Before creating carbonsession, sets as below:</p>
-<pre><code>import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-CarbonProperties.getInstance().addProperty(CarbonCommonConstants.LOCK_TYPE, "HDFSLOCK")
-</code></pre>
-<h2>
-<a id="failed-to-load-thrift-libraries" class="anchor" href="#failed-to-load-thrift-libraries" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to load thrift libraries</h2>
-<p><strong>Symptom</strong></p>
-<p>Thrift throws following exception :</p>
-<pre><code>thrift: error while loading shared libraries:
-libthriftc.so.0: cannot open shared object file: No such file or directory
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The complete path to the directory containing the libraries is not configured correctly.</p>
-<p><strong>Procedure</strong></p>
-<p>Follow the Apache thrift docs at <a href="https://thrift.apache.org/docs/install" target=_blank rel="nofollow">https://thrift.apache.org/docs/install</a> to install thrift correctly.</p>
-<h2>
-<a id="failed-to-launch-the-spark-shell" class="anchor" href="#failed-to-launch-the-spark-shell" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to launch the Spark Shell</h2>
-<p><strong>Symptom</strong></p>
-<p>The shell prompts the following error :</p>
-<pre><code>org.apache.spark.sql.CarbonContext$$anon$$apache$spark$sql$catalyst$analysis
-$OverrideCatalog$_setter_$org$apache$spark$sql$catalyst$analysis
-$OverrideCatalog$$overrides_$e
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The Spark Version and the selected Spark Profile do not match.</p>
-<p><strong>Procedure</strong></p>
-<ol>
-<li>
-<p>Ensure your spark version and selected profile for spark are correct.</p>
-</li>
-<li>
-<p>Use the following command :</p>
-</li>
-</ol>
-<pre><code>"mvn -Pspark-2.1 -Dspark.version {yourSparkVersion} clean package"
-</code></pre>
-<p>Note :  Refrain from using "mvn clean package" without specifying the profile.</p>
-<h2>
-<a id="failed-to-execute-load-query-on-cluster" class="anchor" href="#failed-to-execute-load-query-on-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute load query on cluster.</h2>
-<p><strong>Symptom</strong></p>
-<p>Load query failed with the following exception:</p>
-<pre><code>Dictionary file is locked for updation.
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The carbon.properties file is not identical in all the nodes of the cluster.</p>
-<p><strong>Procedure</strong></p>
-<p>Follow the steps to ensure the carbon.properties file is consistent across all the nodes:</p>
-<ol>
-<li>
-<p>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.</p>
-</li>
-<li>
-<p>For the changes to take effect, restart the Spark cluster.</p>
-</li>
-</ol>
-<h2>
-<a id="failed-to-execute-insert-query-on-cluster" class="anchor" href="#failed-to-execute-insert-query-on-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute insert query on cluster.</h2>
-<p><strong>Symptom</strong></p>
-<p>Load query failed with the following exception:</p>
-<pre><code>Dictionary file is locked for updation.
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The carbon.properties file is not identical in all the nodes of the cluster.</p>
-<p><strong>Procedure</strong></p>
-<p>Follow the steps to ensure the carbon.properties file is consistent across all the nodes:</p>
-<ol>
-<li>
-<p>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.</p>
-</li>
-<li>
-<p>For the changes to take effect, restart the Spark cluster.</p>
-</li>
-</ol>
-<h2>
-<a id="failed-to-connect-to-hiveuser-with-thrift" class="anchor" href="#failed-to-connect-to-hiveuser-with-thrift" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to connect to hiveuser with thrift</h2>
-<p><strong>Symptom</strong></p>
-<p>We get the following exception :</p>
-<pre><code>Cannot connect to hiveuser.
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The external process does not have permission to access.</p>
-<p><strong>Procedure</strong></p>
-<p>Ensure that the Hiveuser in mysql must allow its access to the external processes.</p>
-<h2>
-<a id="failed-to-read-the-metastore-db-during-table-creation" class="anchor" href="#failed-to-read-the-metastore-db-during-table-creation" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to read the metastore db during table creation.</h2>
-<p><strong>Symptom</strong></p>
-<p>We get the following exception on trying to connect :</p>
-<pre><code>Cannot read the metastore db
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The metastore db is dysfunctional.</p>
-<p><strong>Procedure</strong></p>
-<p>Remove the metastore db from the carbon.metastore in the Spark Directory.</p>
-<h2>
-<a id="failed-to-load-data-on-the-cluster" class="anchor" href="#failed-to-load-data-on-the-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to load data on the cluster</h2>
-<p><strong>Symptom</strong></p>
-<p>Data loading fails with the following exception :</p>
-<pre><code>Data Load failure exception
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The following issue can cause the failure :</p>
-<ol>
-<li>
-<p>The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.</p>
-</li>
-<li>
-<p>Path to hdfs ddl is not configured correctly in the carbon.properties.</p>
-</li>
-</ol>
-<p><strong>Procedure</strong></p>
-<p>Follow the steps to ensure the following configuration files are consistent across all the nodes:</p>
-<ol>
-<li>
-<p>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.</p>
-<p>Note : Set the path to hdfs ddl in carbon.properties in the master node.</p>
-</li>
-<li>
-<p>For the changes to take effect, restart the Spark cluster.</p>
-</li>
-</ol>
-<h2>
-<a id="failed-to-insert-data-on-the-cluster" class="anchor" href="#failed-to-insert-data-on-the-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to insert data on the cluster</h2>
-<p><strong>Symptom</strong></p>
-<p>Insertion fails with the following exception :</p>
-<pre><code>Data Load failure exception
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The following issue can cause the failure :</p>
-<ol>
-<li>
-<p>The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.</p>
-</li>
-<li>
-<p>Path to hdfs ddl is not configured correctly in the carbon.properties.</p>
-</li>
-</ol>
-<p><strong>Procedure</strong></p>
-<p>Follow the steps to ensure the following configuration files are consistent across all the nodes:</p>
-<ol>
-<li>
-<p>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.</p>
-<p>Note : Set the path to hdfs ddl in carbon.properties in the master node.</p>
-</li>
-<li>
-<p>For the changes to take effect, restart the Spark cluster.</p>
-</li>
-</ol>
-<h2>
-<a id="failed-to-execute-concurrent-operationsloadinsertupdate-on-table-by-multiple-workers" class="anchor" href="#failed-to-execute-concurrent-operationsloadinsertupdate-on-table-by-multiple-workers" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute Concurrent Operations(Load,Insert,Update) on table by multiple workers.</h2>
-<p><strong>Symptom</strong></p>
-<p>Execution fails with the following exception :</p>
-<pre><code>Table is locked for updation.
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>Concurrency not supported.</p>
-<p><strong>Procedure</strong></p>
-<p>Worker must wait for the query execution to complete and the table to release the lock for another query execution to succeed.</p>
-<h2>
-<a id="failed-to-create-a-table-with-a-single-numeric-column" class="anchor" href="#failed-to-create-a-table-with-a-single-numeric-column" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to create a table with a single numeric column.</h2>
-<p><strong>Symptom</strong></p>
-<p>Execution fails with the following exception :</p>
-<pre><code>Table creation fails.
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>Behaviour not supported.</p>
-<p><strong>Procedure</strong></p>
-<p>A single column that can be considered as dimension is mandatory for table creation.</p>
-<script>
-// Show selected style on nav item
-$(function() { $('.b-nav__trouble').addClass('selected'); });
-</script>
-</div>
-</div>
-</div>
-</div>
-<div class="doc-footer">
-    <a href="#top" class="scroll-top">Top</a>
-</div>
-</div>
-</section>
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/usecases.html
----------------------------------------------------------------------
diff --git a/content/usecases.html b/content/usecases.html
new file mode 100644
index 0000000..cb309dd
--- /dev/null
+++ b/content/usecases.html
@@ -0,0 +1,619 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div><h1>
+<a id="use-cases" class="anchor" href="#use-cases" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Use Cases</h1>
+<p>CarbonData is useful in various analytical work loads.Some of the most typical usecases where CarbonData is being used is documented here.</p>
+<p>CarbonData is used for but not limited to</p>
+<ul>
+<li>
+<h3>
+<a id="bank" class="anchor" href="#bank" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Bank</h3>
+<ul>
+<li>fraud detection analysis</li>
+<li>risk profile analysis</li>
+<li>As a zip table to update the daily balance of customers</li>
+</ul>
+</li>
+<li>
+<h3>
+<a id="telecom" class="anchor" href="#telecom" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Telecom</h3>
+<ul>
+<li>Detection of signal anamolies for VIP customers for providing improved customer experience</li>
+<li>Analysis of MR,CHR records of GSM data to determine the tower load at a particular time period and rebalance the tower configuration</li>
+<li>Analysis of access sites, video, screen size, streaming bandwidth, quality to determine the network quality,routing configuration</li>
+</ul>
+</li>
+<li>
+<h3>
+<a id="webinternet" class="anchor" href="#webinternet" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Web/Internet</h3>
+<ul>
+<li>Analysis of page or video being accessed,server loads, streaming quality, screen size</li>
+</ul>
+</li>
+<li>
+<h3>
+<a id="smart-city" class="anchor" href="#smart-city" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Smart City</h3>
+<ul>
+<li>Vehicle tracking analysis</li>
+<li>Unusual behaviour analysis</li>
+</ul>
+</li>
+</ul>
+<p>These use cases can be broadly classified into below categories:</p>
+<ul>
+<li>Full scan/Detailed/Interactive queries</li>
+<li>Aggregation/OLAP BI queries</li>
+<li>Real time Ingestion(Streaming) and queries</li>
+</ul>
+<h2>
+<a id="detailed-queries-in-the-telecom-scenario" class="anchor" href="#detailed-queries-in-the-telecom-scenario" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Detailed Queries in the Telecom scenario</h2>
+<h3>
+<a id="scenario" class="anchor" href="#scenario" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Scenario</h3>
+<p>User wants to analyse all the CHR(Call History Record) and MR(Measurement Records) of the mobile subscribers in order to identify the service failures within 10 secs.Also user wants to run machine learning models on the data to fairly estimate the reasons and time of probable failures and take action ahead to meet the SLA(Service Level Agreements) of VIP customers.</p>
+<h3>
+<a id="challenges" class="anchor" href="#challenges" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Challenges</h3>
+<ul>
+<li>Data incoming rate might vary based on the user concentration at a particular period of time.Hence higher data load speeds are required</li>
+<li>Cluster needs to be well utilised and share the cluster among various applications for better resource consumption and savings</li>
+<li>Queries needs to be interactive.ie., the queries fetch small data and need to be returned in seconds</li>
+<li>Data Loaded into the system every few minutes.</li>
+</ul>
+<h3>
+<a id="solution" class="anchor" href="#solution" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Solution</h3>
+<p>Setup a Hadoop + Spark + CarbonData cluster managed by YARN.</p>
+<p>Proposed the following configurations for CarbonData.(These tunings were proposed before CarbonData introduced SORT_COLUMNS parameter using which the sort order and schema order could be different.)</p>
+<p>Add the frequently used columns to the left of the table definition.Add it in the increasing order of cardinality.It was suggested to keep msisdn,imsi columns in the beginning of the schema.With latest CarbonData, SORT_COLUMNS needs to be configured msisdn,imsi in the beginning.</p>
+<p>Add timestamp column to the right of the schema as it is naturally increasing.</p>
+<p>Create two separate YARN queues for Query and Data Loading.</p>
+<p>Apart from these, the following CarbonData configuration was suggested to be configured in the cluster.</p>
+<table>
+<thead>
+<tr>
+<th>Configuration for</th>
+<th>Parameter</th>
+<th>Value</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>Data Loading</td>
+<td>carbon.graph.rowset.size</td>
+<td>100000</td>
+<td>Based on the size of each row, this determines the memory required during data loading.Higher value leads to increased memory foot print</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.number.of.cores.while.loading</td>
+<td>12</td>
+<td>More cores can improve data loading speed</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.sort.size</td>
+<td>100000</td>
+<td>Number of records to sort at a time.More number of records configured will lead to increased memory foot print</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>table_blocksize</td>
+<td>256</td>
+<td>To efficiently schedule multiple tasks during query</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.sort.intermediate.files.limit</td>
+<td>100</td>
+<td>Increased to 100 as number of cores are more.Can perform merging in backgorund.If less number of files to merge, sort threads would be idle</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.use.local.dir</td>
+<td>TRUE</td>
+<td>yarn application directory will be usually on a single disk.YARN would be configured with multiple disks to be used as temp or to assign randomly to applications.Using the yarn temp directory will allow carbon to use multiple disks and improve IO performance</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.use.multiple.temp.dir</td>
+<td>TRUE</td>
+<td>multiple disks to write sort files will lead to better IO and reduce the IO bottleneck</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.compaction.level.threshold</td>
+<td>6,6</td>
+<td>Since frequent small loads, compacting more segments will give better query results</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.enable.auto.load.merge</td>
+<td>true</td>
+<td>Since data loading is small,auto compacting keeps the number of segments less and also compaction can complete in  time</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.number.of.cores.while.compacting</td>
+<td>4</td>
+<td>Higher number of cores can improve the compaction speed</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.major.compaction.size</td>
+<td>921600</td>
+<td>Sum of several loads to combine into single segment</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="results-achieved" class="anchor" href="#results-achieved" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Results Achieved</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Results</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>Query</td>
+<td>&lt; 3 Sec</td>
+</tr>
+<tr>
+<td>Data Loading Speed</td>
+<td>40 MB/s Per Node</td>
+</tr>
+<tr>
+<td>Concurrent query performance (20 queries)</td>
+<td>&lt; 10 Sec</td>
+</tr>
+</tbody>
+</table>
+<h2>
+<a id="detailed-queries-in-the-smart-city-scenario" class="anchor" href="#detailed-queries-in-the-smart-city-scenario" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Detailed Queries in the Smart City scenario</h2>
+<h3>
+<a id="scenario-1" class="anchor" href="#scenario-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Scenario</h3>
+<p>User wants to analyse the person/vehicle movement and behavior during a certain time period.This output data needs to be joined with a external table for Human details extraction.The query will be run with different time period as filter to identify potential behavior mismatch.</p>
+<h3>
+<a id="challenges-1" class="anchor" href="#challenges-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Challenges</h3>
+<p>Data generated per day is very huge.Data needs to be loaded multiple times per day to accomodate the incoming data size.</p>
+<p>Data Loading done once in 6 hours.</p>
+<h3>
+<a id="solution-1" class="anchor" href="#solution-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Solution</h3>
+<p>Setup a Hadoop + Spark + CarbonData cluster managed by YARN.</p>
+<p>Since data needs to be queried for a time period, it was recommended to keep the time column at the beginning of schema.</p>
+<p>Use table block size as 512MB.</p>
+<p>Use local sort mode.</p>
+<p>Apart from these, the following CarbonData configuration was suggested to be configured in the cluster.</p>
+<p>Use all columns are no-dictionary as the cardinality is high.</p>
+<table>
+<thead>
+<tr>
+<th>Configuration for</th>
+<th>Parameter</th>
+<th>Value</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>Data Loading</td>
+<td>carbon.graph.rowset.size</td>
+<td>100000</td>
+<td>Based on the size of each row, this determines the memory required during data loading.Higher value leads to increased memory foot print</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>enable.unsafe.sort</td>
+<td>TRUE</td>
+<td>Temporary data generated during sort is huge which causes GC bottlenecks.Using unsafe reduces the pressure on GC</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>enable.offheap.sort</td>
+<td>TRUE</td>
+<td>Temporary data generated during sort is huge which causes GC bottlenecks.Using offheap reduces the pressure on GC.offheap can be accessed through java unsafe.hence enable.unsafe.sort needs to be true</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>offheap.sort.chunk.size.in.mb</td>
+<td>128</td>
+<td>Size of memory to allocate for sorting.Can increase this based on the memory available</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.number.of.cores.while.loading</td>
+<td>12</td>
+<td>Higher cores can improve data loading speed</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.sort.size</td>
+<td>100000</td>
+<td>Number of records to sort at a time.More number of records configured will lead to increased memory foot print</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>table_blocksize</td>
+<td>512</td>
+<td>To efficiently schedule multiple tasks during query.This size depends on data scenario.If data is such that the filters would select less number of blocklets to scan, keeping higher number works well.If the number blocklets to scan is more, better to reduce the size as more tasks can be scheduled in parallel.</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.sort.intermediate.files.limit</td>
+<td>100</td>
+<td>Increased to 100 as number of cores are more.Can perform merging in backgorund.If less number of files to merge, sort threads would be idle</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.use.local.dir</td>
+<td>TRUE</td>
+<td>yarn application directory will be usually on a single disk.YARN would be configured with multiple disks to be used as temp or to assign randomly to applications.Using the yarn temp directory will allow carbon to use multiple disks and improve IO performance</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.use.multiple.temp.dir</td>
+<td>TRUE</td>
+<td>multiple disks to write sort files will lead to better IO and reduce the IO bottleneck</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>sort.inmemory.size.in.mb</td>
+<td>92160</td>
+<td>Memory allocated to do inmemory sorting.When more memory is available in the node, configuring this will retain more sort blocks in memory so that the merge sort is faster due to no/very less IO</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.major.compaction.size</td>
+<td>921600</td>
+<td>Sum of several loads to combine into single segment</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.number.of.cores.while.compacting</td>
+<td>12</td>
+<td>Higher number of cores can improve the compaction speed.Data size is huge.Compaction need to use more threads to speed up the process</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.enable.auto.load.merge</td>
+<td>FALSE</td>
+<td>Doing auto minor compaction is costly process as data size is huge.Perform manual compaction when  the cluster is less loaded</td>
+</tr>
+<tr>
+<td>Query</td>
+<td>carbon.enable.vector.reader</td>
+<td>true</td>
+<td>To fetch results faster, supporting spark vector processing will speed up the query</td>
+</tr>
+<tr>
+<td>Query</td>
+<td>enable.unsafe.in.query.procressing</td>
+<td>true</td>
+<td>Data that needs to be scanned in huge which in turn generates more short lived Java objects.This cause pressure of GC.using unsafe and offheap will reduce the GC overhead</td>
+</tr>
+<tr>
+<td>Query</td>
+<td>use.offheap.in.query.processing</td>
+<td>true</td>
+<td>Data that needs to be scanned in huge which in turn generates more short lived Java objects.This cause pressure of GC.using unsafe and offheap will reduce the GC overhead.offheap can be accessed through java unsafe.hence enable.unsafe.in.query.procressing needs to be true</td>
+</tr>
+<tr>
+<td>Query</td>
+<td>enable.unsafe.columnpage</td>
+<td>TRUE</td>
+<td>Keep the column pages in offheap memory so that the memory overhead due to java object is less and also reduces GC pressure.</td>
+</tr>
+<tr>
+<td>Query</td>
+<td>carbon.unsafe.working.memory.in.mb</td>
+<td>10240</td>
+<td>Amount of memory to use for offheap operations.Can increase this memory based on the data size</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="results-achieved-1" class="anchor" href="#results-achieved-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Results Achieved</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Results</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>Query (Time Period spanning 1 segment)</td>
+<td>&lt; 10 Sec</td>
+</tr>
+<tr>
+<td>Data Loading Speed</td>
+<td>45 MB/s Per Node</td>
+</tr>
+</tbody>
+</table>
+<h2>
+<a id="olapbi-queries-in-the-webinternet-scenario" class="anchor" href="#olapbi-queries-in-the-webinternet-scenario" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>OLAP/BI Queries in the web/Internet scenario</h2>
+<h3>
+<a id="scenario-2" class="anchor" href="#scenario-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Scenario</h3>
+<p>An Internet company wants to analyze the average download speed, kind of handsets used in a particular region/area,kind of Apps being used, what kind of videos are trending in a particular region to enable them to identify the appropriate resolution size of videos to speed up transfer, and perform many more analysis to serve th customers better.</p>
+<h3>
+<a id="challenges-2" class="anchor" href="#challenges-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Challenges</h3>
+<p>Since data is being queried by a BI tool, all the queries contain group by, which means CarbonData need to return more records as limit cannot be pushed down to carbondata layer.</p>
+<p>Results have to be returned faster as the BI tool would not respond till the data is fetched, causing bad user experience.</p>
+<p>Data might be loaded less frequently(once or twice in a day), but raw data size is huge, which causes the group by queries to run slower.</p>
+<p>Concurrent queries can be more due to the BI dashboard</p>
+<h3>
+<a id="goal" class="anchor" href="#goal" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Goal</h3>
+<ol>
+<li>Aggregation queries are faster</li>
+<li>Concurrency is high(Number of concurrent queries supported)</li>
+</ol>
+<h3>
+<a id="solution-2" class="anchor" href="#solution-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Solution</h3>
+<ul>
+<li>Use table block size as 128MB so that pruning is more effective</li>
+<li>Use global sort mode so that the data to be fetched are grouped together</li>
+<li>Create pre-aggregate tables for non timestamp based group by queries</li>
+<li>For queries containing group by date, create timeseries based Datamap(pre-aggregate) tables so that the data is rolled up during creation and fetch is faster</li>
+<li>Reduce the Spark shuffle partitions.(In our configuration on 14 node cluster, it was reduced to 35 from default of 200)</li>
+<li>Enable global dictionary for columns which have less cardinalities.Aggregation can be done on encoded data, there by improving the performance</li>
+<li>For columns whose cardinality is high,enable the local dictionary so that store size is less and can take dictionary benefit for scan</li>
+</ul>
+<h2>
+<a id="handling-near-realtime-data-ingestion-scenario" class="anchor" href="#handling-near-realtime-data-ingestion-scenario" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Handling near realtime data ingestion scenario</h2>
+<h3>
+<a id="scenario-3" class="anchor" href="#scenario-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Scenario</h3>
+<p>Need to support storing of continously arriving data and make it available immediately for query.</p>
+<h3>
+<a id="challenges-3" class="anchor" href="#challenges-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Challenges</h3>
+<p>When the data ingestion is near real time and the data needs to be available for query immediately, usual scenario is to do data loading in micro batches.But this causes the problem of generating many small files.This poses two problems:</p>
+<ol>
+<li>Small file handling in HDFS is inefficient</li>
+<li>CarbonData will suffer in query performance as all the small files will have to be queried when filter is on non time column</li>
+</ol>
+<p>CarbonData will suffer in query performance as all the small files will have to be queried when filter is on non time column.</p>
+<p>Since data is continouly arriving, allocating resources for compaction might not be feasible.</p>
+<h3>
+<a id="goal-1" class="anchor" href="#goal-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Goal</h3>
+<ol>
+<li>Data is available in near real time for query as it arrives</li>
+<li>CarbonData doesnt suffer from small files problem</li>
+</ol>
+<h3>
+<a id="solution-3" class="anchor" href="#solution-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Solution</h3>
+<ul>
+<li>Use Streaming tables support of CarbonData</li>
+<li>Configure the carbon.streaming.segment.max.size property to higher value(default is 1GB) if a bit slower query performance is not a concern</li>
+<li>Configure carbon.streaming.auto.handoff.enabled to true so that after the  carbon.streaming.segment.max.size is reached, the segment is converted into format optimized for query</li>
+<li>Disable auto compaction.Manually trigger the minor compaction with default 4,3 when the cluster is not busy</li>
+<li>Manually trigger Major compaction based on the size of segments and the frequency with which the segments are being created</li>
+<li>Enable local dictionary</li>
+</ul>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__uses').addClass('selected'); });
+</script></div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/useful-tips-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/useful-tips-on-carbondata.html b/content/useful-tips-on-carbondata.html
deleted file mode 100644
index eba1b0a..0000000
--- a/content/useful-tips-on-carbondata.html
+++ /dev/null
@@ -1,542 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
-                                   target="_blank">Apache CarbonData 1.4.1</a></li>
-							<li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
-                                   target="_blank">Apache CarbonData 1.4.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
-                                   target="_blank">Apache CarbonData 1.3.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
-                                   target="_blank">Apache CarbonData 1.3.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="documentation.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search" target="_blank">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input"  placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="verticalnavbar">
-                <nav class="b-sticky-nav">
-                    <div class="nav-scroller">
-                        <div class="nav__inner">
-                            <a class="b-nav__home nav__item" href="/">home</a>
-                            <a class="b-nav__intro nav__item" href="/intro">introduction</a>
-                            <a class="b-nav__quickstart nav__item" href="/quickstart">quickstart</a>
-                            <a class="b-nav__uses nav__item" href="/uses">use cases</a>
-
-                            <div class="nav__item nav__item__with__subs">
-                                <a class="b-nav__docs nav__item nav__sub__anchor" href="/documentation">documentation</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#gettingStarted">getting started</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#api">APIs</a>
-                                <a class="b-nav__streams nav__item nav__sub__item" href="/documentation/streams">kafka streams</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#connect">kafka connect</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#configuration">configuration</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#design">design</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#implementation">implementation</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#operations">operations</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#security">security</a>
-                            </div>
-
-                            <a class="b-nav__performance nav__item" href="/performance">performance</a>
-                            <a class="b-nav__poweredby nav__item" href="/powered-by">powered by</a>
-                            <a class="b-nav__project nav__item" href="/project">project info</a>
-                            <a class="b-nav__ecosystem nav__item" href="https://cwiki.apache.org/confluence/display/KAFKA/Ecosystem" target="_blank">ecosystem</a>
-                            <a class="b-nav__clients nav__item" href="https://cwiki.apache.org/confluence/display/KAFKA/Clients" target="_blank">clients</a>
-                            <a class="b-nav__events nav__item" href="/events">events</a>
-                            <a class="b-nav__contact nav__item" href="/contact">contact us</a>
-
-                            <div class="nav__item nav__item__with__subs">
-                                <a class="b-nav__apache nav__item nav__sub__anchor b-nav__sub__anchor" href="#">apache</a>
-                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/" target="_blank">foundation</a>
-                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/licenses/" target="_blank">license</a>
-                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/foundation/sponsorship.html" target="_blank">sponsorship</a>
-                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/foundation/thanks.html" target="_blank">thanks</a>
-                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/security/" target="_blank">security</a>
-                            </div>
-
-                            <a class="btn" href="/downloads">download</a>
-                            <div class="social-links">
-                                <a class="twitter" href="https://twitter.com/apachekafka" target="_blank">@apachekafka</a>
-                            </div>
-                        </div>
-                    </div>
-                    <div class="navindicator">
-                        <div class="b-nav__home navindicator__item"></div>
-                        <div class="b-nav__intro navindicator__item"></div>
-                        <div class="b-nav__quickstart navindicator__item"></div>
-                        <div class="b-nav__uses navindicator__item"></div>
-                        <div class="b-nav__docs navindicator__item"></div>
-                        <div class="b-nav__performance navindicator__item"></div>
-                        <div class="b-nav__poweredby navindicator__item"></div>
-                        <div class="b-nav__project navindicator__item"></div>
-                        <div class="b-nav__ecosystem navindicator__item"></div>
-                        <div class="b-nav__clients navindicator__item"></div>
-                        <div class="b-nav__events navindicator__item"></div>
-                        <div class="b-nav__contact navindicator__item"></div>
-                    </div>
-                </nav>
-            </div>
-            <div class="mdcontent">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="row">
-                                <div class="col-sm-12  col-md-12">
-                                    <div>
-<h1>
-<a id="useful-tips" class="anchor" href="#useful-tips" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Useful Tips</h1>
-<p>This tutorial guides you to create CarbonData Tables and optimize performance.
-The following sections will elaborate on the below topics :</p>
-<ul>
-<li><a href="#suggestions-to-create-carbondata-table">Suggestions to create CarbonData Table</a></li>
-<li><a href="#configuration-for-optimizing-data-loading-performance-for-massive-data">Configuration for Optimizing Data Loading performance for Massive Data</a></li>
-<li><a href="#configurations-for-optimizing-carbondata-performance">Optimizing Mass Data Loading</a></li>
-</ul>
-<h2>
-<a id="suggestions-to-create-carbondata-table" class="anchor" href="#suggestions-to-create-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Suggestions to Create CarbonData Table</h2>
-<p>For example, the results of the analysis for table creation with dimensions ranging from 10 thousand to 10 billion rows and 100 to 300 columns have been summarized below.
-The following table describes some of the columns from the table used.</p>
-<ul>
-<li><strong>Table Column Description</strong></li>
-</ul>
-<table>
-<thead>
-<tr>
-<th>Column Name</th>
-<th>Data Type</th>
-<th>Cardinality</th>
-<th>Attribution</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>msisdn</td>
-<td>String</td>
-<td>30 million</td>
-<td>Dimension</td>
-</tr>
-<tr>
-<td>BEGIN_TIME</td>
-<td>BigInt</td>
-<td>10 Thousand</td>
-<td>Dimension</td>
-</tr>
-<tr>
-<td>HOST</td>
-<td>String</td>
-<td>1 million</td>
-<td>Dimension</td>
-</tr>
-<tr>
-<td>Dime_1</td>
-<td>String</td>
-<td>1 Thousand</td>
-<td>Dimension</td>
-</tr>
-<tr>
-<td>counter_1</td>
-<td>Decimal</td>
-<td>NA</td>
-<td>Measure</td>
-</tr>
-<tr>
-<td>counter_2</td>
-<td>Numeric(20,0)</td>
-<td>NA</td>
-<td>Measure</td>
-</tr>
-<tr>
-<td>...</td>
-<td>...</td>
-<td>NA</td>
-<td>Measure</td>
-</tr>
-<tr>
-<td>counter_100</td>
-<td>Decimal</td>
-<td>NA</td>
-<td>Measure</td>
-</tr>
-</tbody>
-</table>
-<ul>
-<li><strong>Put the frequently-used column filter in the beginning</strong></li>
-</ul>
-<p>For example, MSISDN filter is used in most of the query then we must put the MSISDN in the first column.
-The create table command can be modified as suggested below :</p>
-<pre><code>create table carbondata_table(
-  msisdn String,
-  BEGIN_TIME bigint,
-  HOST String,
-  Dime_1 String,
-  counter_1, Decimal
-  ...
-  
-  )STORED BY 'carbondata'
-  TBLPROPERTIES ('SORT_COLUMNS'='msisdn, Dime_1')
-</code></pre>
-<p>Now the query with MSISDN in the filter will be more efficient.</p>
-<ul>
-<li><strong>Put the frequently-used columns in the order of low to high cardinality</strong></li>
-</ul>
-<p>If the table in the specified query has multiple columns which are frequently used to filter the results, it is suggested to put
-the columns in the order of cardinality low to high. This ordering of frequently used columns improves the compression ratio and
-enhances the performance of queries with filter on these columns.</p>
-<p>For example, if MSISDN, HOST and Dime_1 are frequently-used columns, then the column order of table is suggested as
-Dime_1&gt;HOST&gt;MSISDN, because Dime_1 has the lowest cardinality.
-The create table command can be modified as suggested below :</p>
-<pre><code>create table carbondata_table(
-    msisdn String,
-    BEGIN_TIME bigint,
-    HOST String,
-    Dime_1 String,
-    counter_1, Decimal
-    ...
-    
-    )STORED BY 'carbondata'
-    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
-</code></pre>
-<ul>
-<li><strong>For measure type columns with non high accuracy, replace Numeric(20,0) data type with Double data type</strong></li>
-</ul>
-<p>For columns of measure type, not requiring high accuracy, it is suggested to replace Numeric data type with Double to enhance query performance.
-The create table command can be modified as below :</p>
-<pre><code>  create table carbondata_table(
-    Dime_1 String,
-    BEGIN_TIME bigint,
-    END_TIME bigint,
-    HOST String,
-    MSISDN String,
-    counter_1 decimal,
-    counter_2 double,
-    ...
-    )STORED BY 'carbondata'
-    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
-</code></pre>
-<p>The result of performance analysis of test-case shows reduction in query execution time from 15 to 3 seconds, thereby improving performance by nearly 5 times.</p>
-<ul>
-<li><strong>Columns of incremental character should be re-arranged at the end of dimensions</strong></li>
-</ul>
-<p>Consider the following scenario where data is loaded each day and the begin_time is incremental for each load, it is suggested to put begin_time at the end of dimensions.
-Incremental values are efficient in using min/max index. The create table command can be modified as below :</p>
-<pre><code>create table carbondata_table(
-  Dime_1 String,
-  HOST String,
-  MSISDN String,
-  counter_1 double,
-  counter_2 double,
-  BEGIN_TIME bigint,
-  END_TIME bigint,
-  ...
-  counter_100 double
-  )STORED BY 'carbondata'
-  TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>BloomFilter can be created to enhance performance for queries with precise equal/in conditions. You can find more information about it in BloomFilter datamap <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/bloomfilter-datamap-guide.html" target=_blank>document</a>.</li>
-</ul>
-<h2>
-<a id="configuration-for-optimizing-data-loading-performance-for-massive-data" class="anchor" href="#configuration-for-optimizing-data-loading-performance-for-massive-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configuration for Optimizing Data Loading performance for Massive Data</h2>
-<p>CarbonData supports large data load, in this process sorting data while loading consumes a lot of memory and disk IO and
-this can result sometimes in "Out Of Memory" exception.
-If you do not have much memory to use, then you may prefer to slow the speed of data loading instead of data load failure.
-You can configure CarbonData by tuning following properties in carbon.properties file to get a better performance.</p>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Default Value</th>
-<th>Description/Tuning</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>carbon.number.of.cores.while.loading</td>
-<td>Default: 2.This value should be &gt;= 2</td>
-<td>Specifies the number of cores used for data processing during data loading in CarbonData.</td>
-</tr>
-<tr>
-<td>carbon.sort.size</td>
-<td>Default: 100000. The value should be &gt;= 100.</td>
-<td>Threshold to write local file in sort step when loading data</td>
-</tr>
-<tr>
-<td>carbon.sort.file.write.buffer.size</td>
-<td>Default:  50000.</td>
-<td>DataOutputStream buffer.</td>
-</tr>
-<tr>
-<td>carbon.number.of.cores.block.sort</td>
-<td>Default: 7</td>
-<td>If you have huge memory and CPUs, increase it as you will</td>
-</tr>
-<tr>
-<td>carbon.merge.sort.reader.thread</td>
-<td>Default: 3</td>
-<td>Specifies the number of cores used for temp file merging during data loading in CarbonData.</td>
-</tr>
-<tr>
-<td>carbon.merge.sort.prefetch</td>
-<td>Default: true</td>
-<td>You may want set this value to false if you have not enough memory</td>
-</tr>
-</tbody>
-</table>
-<p>For example, if there are 10 million records, and i have only 16 cores, 64GB memory, will be loaded to CarbonData table.
-Using the default configuration  always fail in sort step. Modify carbon.properties as suggested below:</p>
-<pre><code>carbon.number.of.cores.block.sort=1
-carbon.merge.sort.reader.thread=1
-carbon.sort.size=5000
-carbon.sort.file.write.buffer.size=5000
-carbon.merge.sort.prefetch=false
-</code></pre>
-<h2>
-<a id="configurations-for-optimizing-carbondata-performance" class="anchor" href="#configurations-for-optimizing-carbondata-performance" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configurations for Optimizing CarbonData Performance</h2>
-<p>Recently we did some performance POC on CarbonData for Finance and telecommunication Field. It involved detailed queries and aggregation
-scenarios. After the completion of POC, some of the configurations impacting the performance have been identified and tabulated below :</p>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Location</th>
-<th>Used For</th>
-<th>Description</th>
-<th>Tuning</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>carbon.sort.intermediate.files.limit</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>During the loading of data, local temp is used to sort the data. This number specifies the minimum number of intermediate files after which the  merge sort has to be initiated.</td>
-<td>Increasing the parameter to a higher value will improve the load performance. For example, when we increase the value from 20 to 100, it increases the data load performance from 35MB/S to more than 50MB/S. Higher values of this parameter consumes  more memory during the load.</td>
-</tr>
-<tr>
-<td>carbon.number.of.cores.while.loading</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>Specifies the number of cores used for data processing during data loading in CarbonData.</td>
-<td>If you have more number of CPUs, then you can increase the number of CPUs, which will increase the performance. For example if we increase the value from 2 to 4 then the CSV reading performance can increase about 1 times</td>
-</tr>
-<tr>
-<td>carbon.compaction.level.threshold</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading and Querying</td>
-<td>For minor compaction, specifies the number of segments to be merged in stage 1 and number of compacted segments to be merged in stage 2.</td>
-<td>Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance. Configuring this parameter will merge the small segment to one big segment which will sort the data and improve the performance. For Example in one telecommunication scenario, the performance improves about 2 times after minor compaction.</td>
-</tr>
-<tr>
-<td>spark.sql.shuffle.partitions</td>
-<td>spark/conf/spark-defaults.conf</td>
-<td>Querying</td>
-<td>The number of task started when spark shuffle.</td>
-<td>The value can be 1 to 2 times as much as the executor cores. In an aggregation scenario, reducing the number from 200 to 32 reduced the query time from 17 to 9 seconds.</td>
-</tr>
-<tr>
-<td>spark.executor.instances/spark.executor.cores/spark.executor.memory</td>
-<td>spark/conf/spark-defaults.conf</td>
-<td>Querying</td>
-<td>The number of executors, CPU cores, and memory used for CarbonData query.</td>
-<td>In the bank scenario, we provide the 4 CPUs cores and 15 GB for each executor which can get good performance. This 2 value does not mean more the better. It needs to be configured properly in case of limited resources. For example, In the bank scenario, it has enough CPU 32 cores each node but less memory 64 GB each node. So we cannot give more CPU but less memory. For example, when 4 cores and 12GB for each executor. It sometimes happens GC during the query which impact the query performance very much from the 3 second to more than 15 seconds. In this scenario need to increase the memory or decrease the CPU cores.</td>
-</tr>
-<tr>
-<td>carbon.detail.batch.size</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>The buffer size to store records, returned from the block scan.</td>
-<td>In limit scenario this parameter is very important. For example your query limit is 1000. But if we set this value to 3000 that means we get 3000 records from scan but spark will only take 1000 rows. So the 2000 remaining are useless. In one Finance test case after we set it to 100, in the limit 1000 scenario the performance increase about 2 times in comparison to if we set this value to 12000.</td>
-</tr>
-<tr>
-<td>carbon.use.local.dir</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>Whether use YARN local directories for multi-table load disk load balance</td>
-<td>If this is set it to true CarbonData will use YARN local directories for multi-table load disk load balance, that will improve the data load performance.</td>
-</tr>
-<tr>
-<td>carbon.use.multiple.temp.dir</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>Whether to use multiple YARN local directories during table data loading for disk load balance</td>
-<td>After enabling 'carbon.use.local.dir', if this is set to true, CarbonData will use all YARN local directories during data load for disk load balance, that will improve the data load performance. Please enable this property when you encounter disk hotspot problem during data loading.</td>
-</tr>
-<tr>
-<td>carbon.sort.temp.compressor</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>Specify the name of compressor to compress the intermediate sort temporary files during sort procedure in data loading.</td>
-<td>The optional values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files. This parameter will be useful if you encounter disk bottleneck.</td>
-</tr>
-<tr>
-<td>carbon.load.skewedDataOptimization.enabled</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>Whether to enable size based block allocation strategy for data loading.</td>
-<td>When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data -- It's useful if the size of your input data files varies widely, say 1MB~1GB.</td>
-</tr>
-<tr>
-<td>carbon.load.min.size.enabled</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>Whether to enable node minumun input data size allocation strategy for data loading.</td>
-<td>When loading, carbondata will use node minumun input data size allocation strategy for task distribution. It will make sure the node load the minimum amount of data -- It's useful if the size of your input data files very small, say 1MB~256MB,Avoid generating a large number of small files.</td>
-</tr>
-</tbody>
-</table>
-<p>Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.</p>
-</div>
-</div>
-</div>
-</div>
-<div class="doc-footer">
-    <a href="#top" class="scroll-top">Top</a>
-</div>
-</div>
-</section>
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/videogallery.html
----------------------------------------------------------------------
diff --git a/content/videogallery.html b/content/videogallery.html
index e494174..c282364 100644
--- a/content/videogallery.html
+++ b/content/videogallery.html
@@ -246,4 +246,4 @@
 
 
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/src/main/resources/application.conf b/src/main/resources/application.conf
index af4f234..ba425e5 100644
--- a/src/main/resources/application.conf
+++ b/src/main/resources/application.conf
@@ -15,7 +15,8 @@ fileList=["configuration-parameters",
   "segment-management-on-carbondata",
   "release-guide",
   "how-to-contribute-to-apache-carbondata",
-  "introduction"
+  "introduction",
+  "usecases"
   ]
 dataMapFileList=[
   "bloomfilter-datamap-guide",
@@ -34,4 +35,20 @@ outputFileLocation="src/main/webapp/"
 apiUrl="https://raw.githubusercontent.com/apache/carbondata/master/docs/"
 mdLink="https://api.github.com/markdown/raw"
 mdFileLocation="src/site/markdown/"
-imagesFilesList=["file-structure-of-carbondata"]
\ No newline at end of file
+imagesFilesList=["file-structure-of-carbondata",
+"2-1_1",
+"2-2_1",
+"2-3_1",
+"2-3_2",
+"2-3_3",
+"2-3_4",
+"2-4_1",
+"2-5_1",
+"2-5_2",
+"2-5_3",
+"2-6_1",
+"carbon_data_file_structure_new",
+"carbon_data_format_new",
+"carbondata-performance"
+]
+scriptsPath="src/main/scala/scripts/"
\ No newline at end of file


[29/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/quick-start-guide.html
----------------------------------------------------------------------
diff --git a/content/quick-start-guide.html b/content/quick-start-guide.html
index ea88086..f3e8a8f 100644
--- a/content/quick-start-guide.html
+++ b/content/quick-start-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -217,10 +217,10 @@
 <a id="prerequisites" class="anchor" href="#prerequisites" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h2>
 <ul>
 <li>
-<p>Spark 2.2.1 version is installed and running.CarbonData supports Spark versions upto 2.2.1.Please follow steps described in <a href="https://spark.apache.org/docs/latest" target=_blank rel="nofollow">Spark docs website</a> for installing and running Spark.</p>
+<p>CarbonData supports Spark versions upto 2.2.1.Please download Spark package from <a href="https://spark.apache.org/downloads.html" target=_blank rel="nofollow">Spark website</a></p>
 </li>
 <li>
-<p>Create a sample.csv file using the following commands. The CSV file is required for loading data into CarbonData.</p>
+<p>Create a sample.csv file using the following commands. The CSV file is required for loading data into CarbonData</p>
 <pre><code>cd carbondata
 cat &gt; sample.csv &lt;&lt; EOF
 id,name,city,age
@@ -232,21 +232,18 @@ EOF
 </li>
 </ul>
 <h2>
-<a id="deployment-modes" class="anchor" href="#deployment-modes" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deployment modes</h2>
+<a id="integration" class="anchor" href="#integration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Integration</h2>
 <p>CarbonData can be integrated with Spark and Presto Execution Engines.The below documentation guides on Installing and Configuring with these execution engines.</p>
 <h3>
 <a id="spark" class="anchor" href="#spark" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Spark</h3>
 <p><a href="#installing-and-configuring-carbondata-to-run-locally-with-spark-shell">Installing and Configuring CarbonData to run locally with Spark Shell</a></p>
 <p><a href="#installing-and-configuring-carbondata-on-standalone-spark-cluster">Installing and Configuring CarbonData on Standalone Spark Cluster</a></p>
 <p><a href="#installing-and-configuring-carbondata-on-spark-on-yarn-cluster">Installing and Configuring CarbonData on Spark on YARN Cluster</a></p>
+<p><a href="#query-execution-using-carbondata-thrift-server">Installing and Configuring CarbonData Thrift Server for Query Execution</a></p>
 <h3>
 <a id="presto" class="anchor" href="#presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Presto</h3>
 <p><a href="#installing-and-configuring-carbondata-on-presto">Installing and Configuring CarbonData on Presto</a></p>
 <h2>
-<a id="querying-data" class="anchor" href="#querying-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Querying Data</h2>
-<p><a href="#query-execution-using-carbondata-thrift-server">Query Execution using CarbonData Thrift Server</a></p>
-<h2></h2>
-<h2>
 <a id="installing-and-configuring-carbondata-to-run-locally-with-spark-shell" class="anchor" href="#installing-and-configuring-carbondata-to-run-locally-with-spark-shell" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData to run locally with Spark Shell</h2>
 <p>Apache Spark Shell provides a simple way to learn the API, as well as a powerful tool to analyze data interactively. Please visit <a href="http://spark.apache.org/docs/latest/" target=_blank rel="nofollow">Apache Spark Documentation</a> for more details on Spark shell.</p>
 <h4>
@@ -275,12 +272,12 @@ import org.apache.spark.sql.CarbonSession._
 <h6>
 <a id="creating-a-table" class="anchor" href="#creating-a-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Creating a Table</h6>
 <pre><code>scala&gt;carbon.sql("CREATE TABLE
-                        IF NOT EXISTS test_table(
-                                  id string,
-                                  name string,
-                                  city string,
-                                  age Int)
-                       STORED BY 'carbondata'")
+                    IF NOT EXISTS test_table(
+                    id string,
+                    name string,
+                    city string,
+                    age Int)
+                  STORED AS carbondata")
 </code></pre>
 <h6>
 <a id="loading-data-to-a-table" class="anchor" href="#loading-data-to-a-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Loading Data to a Table</h6>
@@ -558,12 +555,11 @@ hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
 </code></pre>
 <h2>
 <a id="installing-and-configuring-carbondata-on-presto" class="anchor" href="#installing-and-configuring-carbondata-on-presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Presto</h2>
-<ul>
-<li>
+<p><strong>NOTE:</strong> <strong>CarbonData tables cannot be created nor loaded from Presto.User need to create CarbonData Table and load data into it
+either with <a href="#installing-and-configuring-carbondata-to-run-locally-with-spark-shell">Spark</a> or <a href="./sdk-guide.html">SDK</a>.
+Once the table is created,it can be queried from Presto.</strong></p>
 <h3>
 <a id="installing-presto" class="anchor" href="#installing-presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing Presto</h3>
-</li>
-</ul>
 <ol>
 <li>
 <p>Download the 0.187 version of Presto using:
@@ -684,12 +680,23 @@ discovery.uri=&lt;coordinator_ip&gt;:8086
 <pre><code>select * from system.runtime.nodes;
 </code></pre>
 <p>Now you can use the Presto CLI on the coordinator to query data sources in the catalog using the Presto workers.</p>
+<p>List the schemas(databases) available</p>
+<pre><code>show schemas;
+</code></pre>
+<p>Selected the schema where CarbonData table resides</p>
+<pre><code>use carbonschema;
+</code></pre>
+<p>List the available tables</p>
+<pre><code>show tables;
+</code></pre>
+<p>Query from the available tables</p>
+<pre><code>select * from carbon_table;
+</code></pre>
 <p><strong>Note :</strong> Create Tables and data loads should be done before executing queries as we can not create carbon table from this interface.</p>
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__quickstart').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -704,4 +711,4 @@ $(function() { $('.b-nav__quickstart').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/release-guide.html
----------------------------------------------------------------------
diff --git a/content/release-guide.html b/content/release-guide.html
index fb51368..cb47540 100644
--- a/content/release-guide.html
+++ b/content/release-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -553,8 +553,7 @@ contributions.</p>
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__release').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -569,4 +568,4 @@ $(function() { $('.b-nav__release').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/s3-guide.html
----------------------------------------------------------------------
diff --git a/content/s3-guide.html b/content/s3-guide.html
index 9042b19..57af913 100644
--- a/content/s3-guide.html
+++ b/content/s3-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -277,8 +277,7 @@ to a HDFS directory.</li>
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__s3').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -293,4 +292,4 @@ $(function() { $('.b-nav__s3').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/sdk-guide.html
----------------------------------------------------------------------
diff --git a/content/sdk-guide.html b/content/sdk-guide.html
index 95ae214..a252965 100644
--- a/content/sdk-guide.html
+++ b/content/sdk-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -212,9 +212,14 @@
                                     <div>
 <h1>
 <a id="sdk-guide" class="anchor" href="#sdk-guide" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SDK Guide</h1>
-<p>In the carbon jars package, there exist a carbondata-store-sdk-x.x.x-SNAPSHOT.jar, including SDK writer and reader.</p>
+<p>CarbonData provides SDK to facilitate</p>
+<ol>
+<li><a href="#sdk-writer">Writing carbondata files from other application which does not use Spark</a></li>
+<li><a href="#sdk-reader">Reading carbondata files from other application which does not use Spark</a></li>
+</ol>
 <h1>
 <a id="sdk-writer" class="anchor" href="#sdk-writer" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SDK Writer</h1>
+<p>In the carbon jars package, there exist a carbondata-store-sdk-x.x.x-SNAPSHOT.jar, including SDK writer and reader.</p>
 <p>This SDK writer, writes carbondata file and carbonindex file at a given path.
 External client can make use of this writer to convert other format data or live data to create carbondata and index files.
 These SDK writer output contains just a carbondata and carbonindex files. No metadata folder will be present.</p>
@@ -989,8 +994,7 @@ public String getProperty(String key, String defaultValue);
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__api').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -1005,4 +1009,4 @@ $(function() { $('.b-nav__api').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/sdk-writer-guide.html
----------------------------------------------------------------------
diff --git a/content/sdk-writer-guide.html b/content/sdk-writer-guide.html
deleted file mode 100644
index 5553194..0000000
--- a/content/sdk-writer-guide.html
+++ /dev/null
@@ -1,549 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
-                                   target="_blank">Apache CarbonData 1.4.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
-                                   target="_blank">Apache CarbonData 1.3.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
-                                   target="_blank">Apache CarbonData 1.3.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
-                                   target="_blank">Apache CarbonData 1.2.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
-                                   target="_blank">Apache CarbonData 1.1.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
-                                   target="_blank">Apache CarbonData 1.1.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
-                                   target="_blank">Apache CarbonData 1.0.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
-                                   target="_blank">Apache CarbonData 0.2.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
-                                   target="_blank">Apache CarbonData 0.1.1</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
-                                   target="_blank">Apache CarbonData 0.1.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search" target="_blank">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input"  placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="row">
-                                <div class="col-sm-12  col-md-12">
-                                    <div><h1>
-<a id="sdk-writer-guide" class="anchor" href="#sdk-writer-guide" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SDK Writer Guide</h1>
-<p>In the carbon jars package, there exist a carbondata-store-sdk-x.x.x-SNAPSHOT.jar.
-This SDK writer, writes carbondata file and carbonindex file at a given path.
-External client can make use of this writer to convert other format data or live data to create carbondata and index files.
-These SDK writer output contains just a carbondata and carbonindex files. No metadata folder will be present.</p>
-<h2>
-<a id="quick-example" class="anchor" href="#quick-example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Quick example</h2>
-<h3>
-<a id="example-with-csv-format" class="anchor" href="#example-with-csv-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example with csv format</h3>
-<div class="highlight highlight-source-java"><pre> <span class="pl-k">import</span> <span class="pl-smi">java.io.IOException</span>;
- 
- <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException</span>;
- <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.core.metadata.datatype.DataTypes</span>;
- <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.CarbonWriter</span>;
- <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.CarbonWriterBuilder</span>;
- <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.Field</span>;
- <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.Schema</span>;
- 
- <span class="pl-k">public</span> <span class="pl-k">class</span> <span class="pl-en">TestSdk</span> {
- 
-   <span class="pl-k">public</span> <span class="pl-k">static</span> <span class="pl-k">void</span> <span class="pl-en">main</span>(<span class="pl-k">String</span>[] <span class="pl-v">args</span>) <span class="pl-k">throws</span> <span class="pl-smi">IOException</span>, <span class="pl-smi">InvalidLoadOptionException</span> {
-     testSdkWriter();
-   }
- 
-   <span class="pl-k">public</span> <span class="pl-k">static</span> <span class="pl-k">void</span> <span class="pl-en">testSdkWriter</span>() <span class="pl-k">throws</span> <span class="pl-smi">IOException</span>, <span class="pl-smi">InvalidLoadOptionException</span> {
-     <span class="pl-smi">String</span> path <span class="pl-k">=</span> <span class="pl-s"><span class="pl-pds">"</span>/home/root1/Documents/ab/temp<span class="pl-pds">"</span></span>;
- 
-     <span class="pl-k">Field</span>[] fields <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">Field</span>[<span class="pl-c1">2</span>];
-     fields[<span class="pl-c1">0</span>] <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">Field</span>(<span class="pl-s"><span class="pl-pds">"</span>name<span class="pl-pds">"</span></span>, <span class="pl-smi">DataTypes</span><span class="pl-c1"><span class="pl-k">.</span>STRING</span>);
-     fields[<span class="pl-c1">1</span>] <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">Field</span>(<span class="pl-s"><span class="pl-pds">"</span>age<span class="pl-pds">"</span></span>, <span class="pl-smi">DataTypes</span><span class="pl-c1"><span class="pl-k">.</span>INT</span>);
- 
-     <span class="pl-smi">Schema</span> schema <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">Schema</span>(fields);
- 
-     <span class="pl-smi">CarbonWriterBuilder</span> builder <span class="pl-k">=</span> <span class="pl-smi">CarbonWriter</span><span class="pl-k">.</span>builder()<span class="pl-k">.</span>withSchema(schema)<span class="pl-k">.</span>outputPath(path);
- 
-     <span class="pl-smi">CarbonWriter</span> writer <span class="pl-k">=</span> builder<span class="pl-k">.</span>buildWriterForCSVInput();
- 
-     <span class="pl-k">int</span> rows <span class="pl-k">=</span> <span class="pl-c1">5</span>;
-     <span class="pl-k">for</span> (<span class="pl-k">int</span> i <span class="pl-k">=</span> <span class="pl-c1">0</span>; i <span class="pl-k">&lt;</span> rows; i<span class="pl-k">++</span>) {
-       writer<span class="pl-k">.</span>write(<span class="pl-k">new</span> <span class="pl-smi">String</span>[] { <span class="pl-s"><span class="pl-pds">"</span>robot<span class="pl-pds">"</span></span> <span class="pl-k">+</span> (i <span class="pl-k">%</span> <span class="pl-c1">10</span>), <span class="pl-smi">String</span><span class="pl-k">.</span>valueOf(i) });
-     }
-     writer<span class="pl-k">.</span>close();
-   }
- }</pre></div>
-<h3>
-<a id="example-with-avro-format" class="anchor" href="#example-with-avro-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example with Avro format</h3>
-<div class="highlight highlight-source-java"><pre><span class="pl-k">import</span> <span class="pl-smi">java.io.IOException</span>;
-
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException</span>;
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.core.metadata.datatype.DataTypes</span>;
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.AvroCarbonWriter</span>;
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.CarbonWriter</span>;
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.Field</span>;
-
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.avro.generic.GenericData</span>;
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.commons.lang.CharEncoding</span>;
-
-<span class="pl-k">import</span> <span class="pl-smi">tech.allegro.schema.json2avro.converter.JsonAvroConverter</span>;
-
-<span class="pl-k">public</span> <span class="pl-k">class</span> <span class="pl-en">TestSdkAvro</span> {
-
-  <span class="pl-k">public</span> <span class="pl-k">static</span> <span class="pl-k">void</span> <span class="pl-en">main</span>(<span class="pl-k">String</span>[] <span class="pl-v">args</span>) <span class="pl-k">throws</span> <span class="pl-smi">IOException</span>, <span class="pl-smi">InvalidLoadOptionException</span> {
-    testSdkWriter();
-  }
-
-
-  <span class="pl-k">public</span> <span class="pl-k">static</span> <span class="pl-k">void</span> <span class="pl-en">testSdkWriter</span>() <span class="pl-k">throws</span> <span class="pl-smi">IOException</span>, <span class="pl-smi">InvalidLoadOptionException</span> {
-    <span class="pl-smi">String</span> path <span class="pl-k">=</span> <span class="pl-s"><span class="pl-pds">"</span>./AvroCarbonWriterSuiteWriteFiles<span class="pl-pds">"</span></span>;
-    <span class="pl-c"><span class="pl-c">//</span> Avro schema</span>
-    <span class="pl-smi">String</span> avroSchema <span class="pl-k">=</span>
-        <span class="pl-s"><span class="pl-pds">"</span>{<span class="pl-pds">"</span></span> <span class="pl-k">+</span>
-            <span class="pl-s"><span class="pl-pds">"</span>   <span class="pl-cce">\"</span>type<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>record<span class="pl-cce">\"</span>,<span class="pl-pds">"</span></span> <span class="pl-k">+</span>
-            <span class="pl-s"><span class="pl-pds">"</span>   <span class="pl-cce">\"</span>name<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>Acme<span class="pl-cce">\"</span>,<span class="pl-pds">"</span></span> <span class="pl-k">+</span>
-            <span class="pl-s"><span class="pl-pds">"</span>   <span class="pl-cce">\"</span>fields<span class="pl-cce">\"</span> : [<span class="pl-pds">"</span></span>
-            <span class="pl-k">+</span> <span class="pl-s"><span class="pl-pds">"</span>{ <span class="pl-cce">\"</span>name<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>fname<span class="pl-cce">\"</span>, <span class="pl-cce">\"</span>type<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>string<span class="pl-cce">\"</span> },<span class="pl-pds">"</span></span>
-            <span class="pl-k">+</span> <span class="pl-s"><span class="pl-pds">"</span>{ <span class="pl-cce">\"</span>name<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>age<span class="pl-cce">\"</span>, <span class="pl-cce">\"</span>type<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>int<span class="pl-cce">\"</span> }]<span class="pl-pds">"</span></span> <span class="pl-k">+</span>
-            <span class="pl-s"><span class="pl-pds">"</span>}<span class="pl-pds">"</span></span>;
-
-    <span class="pl-smi">String</span> json <span class="pl-k">=</span> <span class="pl-s"><span class="pl-pds">"</span>{<span class="pl-cce">\"</span>fname<span class="pl-cce">\"</span>:<span class="pl-cce">\"</span>bob<span class="pl-cce">\"</span>, <span class="pl-cce">\"</span>age<span class="pl-cce">\"</span>:10}<span class="pl-pds">"</span></span>;
-
-    <span class="pl-c"><span class="pl-c">//</span> conversion to GenericData.Record</span>
-    <span class="pl-smi">JsonAvroConverter</span> converter <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">JsonAvroConverter</span>();
-    <span class="pl-smi">GenericData</span><span class="pl-k">.</span><span class="pl-smi">Record</span> record <span class="pl-k">=</span> converter<span class="pl-k">.</span>convertToGenericDataRecord(
-        json<span class="pl-k">.</span>getBytes(<span class="pl-smi">CharEncoding</span><span class="pl-c1"><span class="pl-k">.</span>UTF_8</span>), <span class="pl-k">new</span> <span class="pl-smi">org.apache.avro<span class="pl-k">.</span>Schema</span>.<span class="pl-smi">Parser</span>()<span class="pl-k">.</span>parse(avroSchema));
-
-    <span class="pl-c"><span class="pl-c">//</span> prepare carbon schema from avro schema </span>
-    <span class="pl-smi">org.apache.carbondata.sdk.file<span class="pl-k">.</span>Schema</span> carbonSchema <span class="pl-k">=</span>
-            <span class="pl-smi">AvroCarbonWriter</span><span class="pl-k">.</span>getCarbonSchemaFromAvroSchema(avroSchema);
-
-    <span class="pl-k">try</span> {
-      <span class="pl-smi">CarbonWriter</span> writer <span class="pl-k">=</span> <span class="pl-smi">CarbonWriter</span><span class="pl-k">.</span>builder()
-          .withSchema(carbonSchema)
-          .outputPath(path)
-          .buildWriterForAvroInput();
-
-      <span class="pl-k">for</span> (<span class="pl-k">int</span> i <span class="pl-k">=</span> <span class="pl-c1">0</span>; i <span class="pl-k">&lt;</span> <span class="pl-c1">100</span>; i<span class="pl-k">++</span>) {
-        writer<span class="pl-k">.</span>write(record);
-      }
-      writer<span class="pl-k">.</span>close();
-    } <span class="pl-k">catch</span> (<span class="pl-smi">Exception</span> e) {
-      e<span class="pl-k">.</span>printStackTrace();
-    }
-  }
-}</pre></div>
-<h2>
-<a id="datatypes-mapping" class="anchor" href="#datatypes-mapping" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Datatypes Mapping</h2>
-<p>Each of SQL data types are mapped into data types of SDK. Following are the mapping:</p>
-<table>
-<thead>
-<tr>
-<th>SQL DataTypes</th>
-<th>Mapped SDK DataTypes</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>BOOLEAN</td>
-<td>DataTypes.BOOLEAN</td>
-</tr>
-<tr>
-<td>SMALLINT</td>
-<td>DataTypes.SHORT</td>
-</tr>
-<tr>
-<td>INTEGER</td>
-<td>DataTypes.INT</td>
-</tr>
-<tr>
-<td>BIGINT</td>
-<td>DataTypes.LONG</td>
-</tr>
-<tr>
-<td>DOUBLE</td>
-<td>DataTypes.DOUBLE</td>
-</tr>
-<tr>
-<td>VARCHAR</td>
-<td>DataTypes.STRING</td>
-</tr>
-<tr>
-<td>DATE</td>
-<td>DataTypes.DATE</td>
-</tr>
-<tr>
-<td>TIMESTAMP</td>
-<td>DataTypes.TIMESTAMP</td>
-</tr>
-<tr>
-<td>STRING</td>
-<td>DataTypes.STRING</td>
-</tr>
-<tr>
-<td>DECIMAL</td>
-<td>DataTypes.createDecimalType(precision, scale)</td>
-</tr>
-</tbody>
-</table>
-<h2>
-<a id="api-list" class="anchor" href="#api-list" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>API List</h2>
-<h3>
-<a id="class-orgapachecarbondatasdkfilecarbonwriterbuilder" class="anchor" href="#class-orgapachecarbondatasdkfilecarbonwriterbuilder" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.CarbonWriterBuilder</h3>
-<pre><code>/**
-* prepares the builder with the schema provided
-* @param schema is instance of Schema
-*        This method must be called when building CarbonWriterBuilder
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder withSchema(Schema schema);
-</code></pre>
-<pre><code>/**
-* Sets the output path of the writer builder
-* @param path is the absolute path where output files are written
-*             This method must be called when building CarbonWriterBuilder
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder outputPath(String path);
-</code></pre>
-<pre><code>/**
-* If set false, writes the carbondata and carbonindex files in a flat folder structure
-* @param isTransactionalTable is a boolelan value
-*             if set to false, then writes the carbondata and carbonindex files
-*                                                            in a flat folder structure.
-*             if set to true, then writes the carbondata and carbonindex files
-*                                                            in segment folder structure..
-*             By default set to false.
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder isTransactionalTable(boolean isTransactionalTable);
-</code></pre>
-<pre><code>/**
-* to set the timestamp in the carbondata and carbonindex index files
-* @param UUID is a timestamp to be used in the carbondata and carbonindex index files.
-*             By default set to zero.
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder uniqueIdentifier(long UUID);
-</code></pre>
-<pre><code>/**
-* To set the carbondata file size in MB between 1MB-2048MB
-* @param blockSize is size in MB between 1MB to 2048 MB
-*                  default value is 1024 MB
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder withBlockSize(int blockSize);
-</code></pre>
-<pre><code>/**
-* To set the blocklet size of carbondata file
-* @param blockletSize is blocklet size in MB
-*                     default value is 64 MB
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder withBlockletSize(int blockletSize);
-</code></pre>
-<pre><code>/**
-* sets the list of columns that needs to be in sorted order
-* @param sortColumns is a string array of columns that needs to be sorted.
-*                    If it is null or by default all dimensions are selected for sorting
-*                    If it is empty array, no columns are sorted
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder sortBy(String[] sortColumns);
-</code></pre>
-<pre><code>/**
-* If set, create a schema file in metadata folder.
-* @param persist is a boolean value, If set to true, creates a schema file in metadata folder.
-*                By default set to false. will not create metadata folder
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder persistSchemaFile(boolean persist);
-</code></pre>
-<pre><code>/**
-* sets the taskNo for the writer. SDKs concurrently running
-* will set taskNo in order to avoid conflicts in file's name during write.
-* @param taskNo is the TaskNo user wants to specify.
-*               by default it is system time in nano seconds.
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder taskNo(String taskNo);
-</code></pre>
-<pre><code>/**
-* To support the load options for sdk writer
-* @param options key,value pair of load options.
-*                supported keys values are
-*                a. bad_records_logger_enable -- true (write into separate logs), false
-*                b. bad_records_action -- FAIL, FORCE, IGNORE, REDIRECT
-*                c. bad_record_path -- path
-*                d. dateformat -- same as JAVA SimpleDateFormat
-*                e. timestampformat -- same as JAVA SimpleDateFormat
-*                f. complex_delimiter_level_1 -- value to Split the complexTypeData
-*                g. complex_delimiter_level_2 -- value to Split the nested complexTypeData
-*                h. quotechar
-*                i. escapechar
-*
-*                Default values are as follows.
-*
-*                a. bad_records_logger_enable -- "false"
-*                b. bad_records_action -- "FAIL"
-*                c. bad_record_path -- ""
-*                d. dateformat -- "" , uses from carbon.properties file
-*                e. timestampformat -- "", uses from carbon.properties file
-*                f. complex_delimiter_level_1 -- "$"
-*                g. complex_delimiter_level_2 -- ":"
-*                h. quotechar -- "\""
-*                i. escapechar -- "\\"
-*
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder withLoadOptions(Map&lt;String, String&gt; options);
-</code></pre>
-<pre><code>/**
-* Build a {@link CarbonWriter}, which accepts row in CSV format object
-* @return CSVCarbonWriter
-* @throws IOException
-* @throws InvalidLoadOptionException
-*/
-public CarbonWriter buildWriterForCSVInput() throws IOException, InvalidLoadOptionException;
-</code></pre>
-<pre><code>/**
-* Build a {@link CarbonWriter}, which accepts Avro format object
-* @return AvroCarbonWriter 
-* @throws IOException
-* @throws InvalidLoadOptionException
-*/
-public CarbonWriter buildWriterForAvroInput() throws IOException, InvalidLoadOptionException;
-</code></pre>
-<h3>
-<a id="class-orgapachecarbondatasdkfilecarbonwriter" class="anchor" href="#class-orgapachecarbondatasdkfilecarbonwriter" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.CarbonWriter</h3>
-<pre><code>/**
-* Write an object to the file, the format of the object depends on the implementation
-* If AvroCarbonWriter, object is of type org.apache.avro.generic.GenericData.Record 
-* If CSVCarbonWriter, object is of type String[]
-* Note: This API is not thread safe
-* @param object
-* @throws IOException
-*/
-public abstract void write(Object object) throws IOException;
-</code></pre>
-<pre><code>/**
-* Flush and close the writer
-*/
-public abstract void close() throws IOException;
-</code></pre>
-<pre><code>/**
-* Create a {@link CarbonWriterBuilder} to build a {@link CarbonWriter}
-*/
-public static CarbonWriterBuilder builder() {
-return new CarbonWriterBuilder();
-}
-</code></pre>
-<h3>
-<a id="class-orgapachecarbondatasdkfilefield" class="anchor" href="#class-orgapachecarbondatasdkfilefield" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.Field</h3>
-<pre><code>/**
-* Field Constructor
-* @param name name of the field
-* @param type datatype of field, specified in strings.
-*/
-public Field(String name, String type);
-</code></pre>
-<pre><code>/**
-* Field constructor
-* @param name name of the field
-* @param type datatype of the field of class DataType
-*/
-public Field(String name, DataType type);  
-</code></pre>
-<h3>
-<a id="class-orgapachecarbondatasdkfileschema" class="anchor" href="#class-orgapachecarbondatasdkfileschema" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.Schema</h3>
-<pre><code>/**
-* construct a schema with fields
-* @param fields
-*/
-public Schema(Field[] fields);
-</code></pre>
-<pre><code>/**
-* Create a Schema using JSON string, for example:
-* [
-*   {"name":"string"},
-*   {"age":"int"}
-* ] 
-* @param json specified as string
-* @return Schema
-*/
-public static Schema parseJson(String json);
-</code></pre>
-<h3>
-<a id="class-orgapachecarbondatasdkfileavrocarbonwriter" class="anchor" href="#class-orgapachecarbondatasdkfileavrocarbonwriter" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.AvroCarbonWriter</h3>
-<pre><code>/**
-* converts avro schema to carbon schema, required by carbonWriter
-*
-* @param avroSchemaString json formatted avro schema as string
-* @return carbon sdk schema
-*/
-public static org.apache.carbondata.sdk.file.Schema getCarbonSchemaFromAvroSchema(String avroSchemaString);
-</code></pre>
-</div>
-</div>
-</div>
-</div>
-<div class="doc-footer">
-    <a href="#top" class="scroll-top">Top</a>
-</div>
-</div>
-</section>
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/security.html
----------------------------------------------------------------------
diff --git a/content/security.html b/content/security.html
index 2aa4b8e..9168fa3 100644
--- a/content/security.html
+++ b/content/security.html
@@ -205,4 +205,4 @@
 <script src="js/custom.js"></script>
 <script src="js/mdNavigation.js" type="text/javascript"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/segment-management-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/segment-management-on-carbondata.html b/content/segment-management-on-carbondata.html
index 528ee9d..1e6f61d 100644
--- a/content/segment-management-on-carbondata.html
+++ b/content/segment-management-on-carbondata.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -307,8 +307,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -323,4 +322,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/streaming-guide.html
----------------------------------------------------------------------
diff --git a/content/streaming-guide.html b/content/streaming-guide.html
index c6d8391..86f0385 100644
--- a/content/streaming-guide.html
+++ b/content/streaming-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -212,6 +212,37 @@
                                     <div>
 <h1>
 <a id="carbondata-streaming-ingestion" class="anchor" href="#carbondata-streaming-ingestion" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Streaming Ingestion</h1>
+<ul>
+<li>
+<a href="#quick-example">Streaming Table Management</a>
+<ul>
+<li><a href="#create-table-with-streaming-property">Create table with streaming property</a></li>
+<li><a href="#alter-streaming-property">Alter streaming property</a></li>
+<li><a href="#acquire-streaming-lock">Acquire streaming lock</a></li>
+<li><a href="#create-streaming-segment">Create streaming segment</a></li>
+<li><a href="#change-segment-status">Change Stream segment status</a></li>
+<li><a href="#handoff-streaming-finish-segment-to-columnar-segment">Handoff "streaming finish" segment to columnar segment</a></li>
+<li><a href="#auto-handoff-streaming-segment">Auto handoff streaming segment</a></li>
+<li><a href="#stream-data-parser">Stream data parser</a></li>
+<li><a href="#close-streaming-table">Close streaming table</a></li>
+<li><a href="#constraint">Constraints</a></li>
+</ul>
+</li>
+<li>
+<a href="#streamsql">StreamSQL</a>
+<ul>
+<li><a href="#streaming-table">Defining Streaming Table</a></li>
+<li>
+<a href="#streaming-job-management">Streaming Job Management</a>
+<ul>
+<li><a href="#start-stream">START STREAM</a></li>
+<li><a href="#stop-stream">STOP STREAM</a></li>
+<li><a href="#show-streams">SHOW STREAMS</a></li>
+</ul>
+</li>
+</ul>
+</li>
+</ul>
 <h2>
 <a id="quick-example" class="anchor" href="#quick-example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Quick example</h2>
 <p>Download and unzip spark-2.2.0-bin-hadoop2.7.tgz, and export $SPARK_HOME</p>
@@ -255,7 +286,7 @@
 <span class="pl-s">      | col1 INT,</span>
 <span class="pl-s">      | col2 STRING</span>
 <span class="pl-s">      | )</span>
-<span class="pl-s">      | STORED BY 'carbondata'</span>
+<span class="pl-s">      | STORED AS carbondata</span>
 <span class="pl-s">      | TBLPROPERTIES('streaming'='true')<span class="pl-pds">"""</span></span>.stripMargin)
 
  <span class="pl-k">val</span> <span class="pl-smi">carbonTable</span> <span class="pl-k">=</span> <span class="pl-en">CarbonEnv</span>.getCarbonTable(<span class="pl-en">Some</span>(<span class="pl-s"><span class="pl-pds">"</span>default<span class="pl-pds">"</span></span>), <span class="pl-s"><span class="pl-pds">"</span>carbon_table<span class="pl-pds">"</span></span>)(spark)
@@ -300,7 +331,7 @@ streaming table using following DDL.</p>
   col1 <span class="pl-k">INT</span>,
   col2 STRING
  )
- STORED BY <span class="pl-s"><span class="pl-pds">'</span>carbondata<span class="pl-pds">'</span></span>
+ STORED <span class="pl-k">AS</span> carbondata
  TBLPROPERTIES(<span class="pl-s"><span class="pl-pds">'</span>streaming<span class="pl-pds">'</span></span><span class="pl-k">=</span><span class="pl-s"><span class="pl-pds">'</span>true<span class="pl-pds">'</span></span>)</pre></div>
 <table>
 <thead>
@@ -483,6 +514,145 @@ streaming table using following DDL.</p>
 <li>block delete "streaming" segment while the streaming ingestion is running.</li>
 <li>block drop the streaming table while the streaming ingestion is running.</li>
 </ol>
+<h2>
+<a id="streamsql" class="anchor" href="#streamsql" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>StreamSQL</h2>
+<h3>
+<a id="streaming-table" class="anchor" href="#streaming-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Streaming Table</h3>
+<p><strong>Example</strong></p>
+<p>Following example shows how to start a streaming ingest job</p>
+<pre><code>    sql(
+      s"""
+         |CREATE TABLE source(
+         | id INT,
+         | name STRING,
+         | city STRING,
+         | salary FLOAT,
+         | tax DECIMAL(8,2),
+         | percent double,
+         | birthday DATE,
+         | register TIMESTAMP,
+         | updated TIMESTAMP
+         |)
+         |STORED BY carbondata
+         |TBLPROPERTIES (
+         | 'format'='csv',
+         | 'path'='$csvDataDir'
+         |)
+      """.stripMargin)
+
+    sql(
+      s"""
+         |CREATE TABLE sink(
+         | id INT,
+         | name STRING,
+         | city STRING,
+         | salary FLOAT,
+         | tax DECIMAL(8,2),
+         | percent double,
+         | birthday DATE,
+         | register TIMESTAMP,
+         | updated TIMESTAMP
+         |)
+         |STORED BY carbondata
+         |TBLPROPERTIES (
+         |  'streaming'='true'
+         |)
+      """.stripMargin)
+
+    sql(
+      """
+        |START STREAM job123 ON TABLE sink
+        |STMPROPERTIES(
+        |  'trigger'='ProcessingTime',
+        |  'interval'='1 seconds')
+        |AS
+        |  SELECT *
+        |  FROM source
+        |  WHERE id % 2 = 1
+      """.stripMargin)
+
+    sql("STOP STREAM job123")
+
+    sql("SHOW STREAMS [ON TABLE tableName]")
+</code></pre>
+<p>In above example, two table is created: source and sink. The <code>source</code> table's format is <code>csv</code> and <code>sink</code> table format is <code>carbon</code>. Then a streaming job is created to stream data from source table to sink table.</p>
+<p>These two tables are normal carbon table, they can be queried independently.</p>
+<h3>
+<a id="streaming-job-management" class="anchor" href="#streaming-job-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Streaming Job Management</h3>
+<p>As above example shown:</p>
+<ul>
+<li>
+<code>START STREAM jobName ON TABLE tableName</code> is used to start a streaming ingest job.</li>
+<li>
+<code>STOP STREAM jobName</code> is used to stop a streaming job by its name</li>
+<li>
+<code>SHOW STREAMS [ON TABLE tableName]</code> is used to print streaming job information</li>
+</ul>
+<h5>
+<a id="start-stream" class="anchor" href="#start-stream" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>START STREAM</h5>
+<p>When this is issued, carbon will start a structured streaming job to do the streaming ingestion. Before launching the job, system will validate:</p>
+<ul>
+<li>
+<p>The format of table specified in CTAS FROM clause must be one of: csv, json, text, parquet, kafka, socket.  These are formats supported by spark 2.2.0 structured streaming</p>
+</li>
+<li>
+<p>User should pass the options of the streaming source table in its TBLPROPERTIES when creating it. StreamSQL will pass them transparently to spark when creating the streaming job. For example:</p>
+<div class="highlight highlight-source-sql"><pre><span class="pl-k">CREATE</span> <span class="pl-k">TABLE</span> <span class="pl-en">source</span>(
+  name STRING,
+  age <span class="pl-k">INT</span>
+)
+STORED BY carbondata
+TBLPROPERTIES(
+  <span class="pl-s"><span class="pl-pds">'</span>format<span class="pl-pds">'</span></span><span class="pl-k">=</span><span class="pl-s"><span class="pl-pds">'</span>socket<span class="pl-pds">'</span></span>,
+  <span class="pl-s"><span class="pl-pds">'</span>host<span class="pl-pds">'</span></span><span class="pl-k">=</span><span class="pl-s"><span class="pl-pds">'</span>localhost<span class="pl-pds">'</span></span>,
+  <span class="pl-s"><span class="pl-pds">'</span>port<span class="pl-pds">'</span></span><span class="pl-k">=</span><span class="pl-s"><span class="pl-pds">'</span>8888<span class="pl-pds">'</span></span>
+)</pre></div>
+<p>will translate to</p>
+<div class="highlight highlight-source-scala"><pre>spark.readStream
+	 .schema(tableSchema)
+	 .format(<span class="pl-s"><span class="pl-pds">"</span>socket<span class="pl-pds">"</span></span>)
+	 .option(<span class="pl-s"><span class="pl-pds">"</span>host<span class="pl-pds">"</span></span>, <span class="pl-s"><span class="pl-pds">"</span>localhost<span class="pl-pds">"</span></span>)
+	 .option(<span class="pl-s"><span class="pl-pds">"</span>port<span class="pl-pds">"</span></span>, <span class="pl-s"><span class="pl-pds">"</span>8888<span class="pl-pds">"</span></span>)</pre></div>
+</li>
+<li>
+<p>The sink table should have a TBLPROPERTY <code>'streaming'</code> equal to <code>true</code>, indicating it is a streaming table.</p>
+</li>
+<li>
+<p>In the given STMPROPERTIES, user must specify <code>'trigger'</code>, its value must be <code>ProcessingTime</code> (In future, other value will be supported). User should also specify interval value for the streaming job.</p>
+</li>
+<li>
+<p>If the schema specifid in sink table is different from CTAS, the streaming job will fail</p>
+</li>
+</ul>
+<h5>
+<a id="stop-stream" class="anchor" href="#stop-stream" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>STOP STREAM</h5>
+<p>When this is issued, the streaming job will be stopped immediately. It will fail if the jobName specified is not exist.</p>
+<h5>
+<a id="show-streams" class="anchor" href="#show-streams" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW STREAMS</h5>
+<p><code>SHOW STREAMS ON TABLE tableName</code> command will print the streaming job information as following</p>
+<table>
+<thead>
+<tr>
+<th>Job name</th>
+<th>status</th>
+<th>Source</th>
+<th>Sink</th>
+<th>start time</th>
+<th>time elapsed</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>job123</td>
+<td>Started</td>
+<td>device</td>
+<td>fact</td>
+<td>2018-02-03 14:32:42</td>
+<td>10d2h32m</td>
+</tr>
+</tbody>
+</table>
+<p><code>SHOW STREAMS</code> command will show all stream jobs in the system.</p>
 <script>
 $(function() {
   // Show selected style on nav item
@@ -493,8 +663,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -509,4 +678,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/supported-data-types-in-carbondata.html
----------------------------------------------------------------------
diff --git a/content/supported-data-types-in-carbondata.html b/content/supported-data-types-in-carbondata.html
index e0ad647..f346052 100644
--- a/content/supported-data-types-in-carbondata.html
+++ b/content/supported-data-types-in-carbondata.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -269,8 +269,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -285,4 +284,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/timeseries-datamap-guide.html
----------------------------------------------------------------------
diff --git a/content/timeseries-datamap-guide.html b/content/timeseries-datamap-guide.html
index 497b02f..73a4580 100644
--- a/content/timeseries-datamap-guide.html
+++ b/content/timeseries-datamap-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -213,9 +213,9 @@
 <h1>
 <a id="carbondata-timeseries-datamap" class="anchor" href="#carbondata-timeseries-datamap" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Timeseries DataMap</h1>
 <ul>
-<li><a href="#timeseries-datamap-intoduction">Timeseries DataMap Introduction</a></li>
-<li><a href="#compacting-pre-aggregate-tables">Compaction</a></li>
-<li><a href="#data-management-with-pre-aggregate-tables">Data Management</a></li>
+<li><a href="#timeseries-datamap-introduction-alpha-feature">Timeseries DataMap Introduction</a></li>
+<li><a href="#compacting-timeseries-datamp">Compaction</a></li>
+<li><a href="#data-management-on-timeseries-datamap">Data Management</a></li>
 </ul>
 <h2>
 <a id="timeseries-datamap-introduction-alpha-feature" class="anchor" href="#timeseries-datamap-introduction-alpha-feature" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Timeseries DataMap Introduction (Alpha Feature)</h2>
@@ -343,8 +343,7 @@ $(function() {
     $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -359,4 +358,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file


[30/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/installation-guide.html
----------------------------------------------------------------------
diff --git a/content/installation-guide.html b/content/installation-guide.html
deleted file mode 100644
index 696db00..0000000
--- a/content/installation-guide.html
+++ /dev/null
@@ -1,455 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
-                                   target="_blank">Apache CarbonData 1.4.1</a></li>
-							<li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
-                                   target="_blank">Apache CarbonData 1.4.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
-                                   target="_blank">Apache CarbonData 1.3.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
-                                   target="_blank">Apache CarbonData 1.3.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search" target="_blank">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input"  placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="row">
-                                <div class="col-sm-12  col-md-12">
-                                    <div>
-<h1>
-<a id="installation-guide" class="anchor" href="#installation-guide" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installation Guide</h1>
-<p>This tutorial guides you through the installation and configuration of CarbonData in the following two modes :</p>
-<ul>
-<li><a href="#installing-and-configuring-carbondata-on-standalone-spark-cluster">Installing and Configuring CarbonData on Standalone Spark Cluster</a></li>
-<li><a href="#installing-and-configuring-carbondata-on-spark-on-yarn-cluster">Installing and Configuring CarbonData on Spark on YARN Cluster</a></li>
-</ul>
-<p>followed by :</p>
-<ul>
-<li><a href="#query-execution-using-carbondata-thrift-server">Query Execution using CarbonData Thrift Server</a></li>
-</ul>
-<h2>
-<a id="installing-and-configuring-carbondata-on-standalone-spark-cluster" class="anchor" href="#installing-and-configuring-carbondata-on-standalone-spark-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Standalone Spark Cluster</h2>
-<h3>
-<a id="prerequisites" class="anchor" href="#prerequisites" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h3>
-<ul>
-<li>
-<p>Hadoop HDFS and Yarn should be installed and running.</p>
-</li>
-<li>
-<p>Spark should be installed and running on all the cluster nodes.</p>
-</li>
-<li>
-<p>CarbonData user should have permission to access HDFS.</p>
-</li>
-</ul>
-<h3>
-<a id="procedure" class="anchor" href="#procedure" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Procedure</h3>
-<ol>
-<li>
-<p><a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>Build the CarbonData</a> project and get the assembly jar from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code>.</p>
-</li>
-<li>
-<p>Copy <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code> to <code>$SPARK_HOME/carbonlib</code> folder.</p>
-<p><strong>NOTE</strong>: Create the carbonlib folder if it does not exist inside <code>$SPARK_HOME</code> path.</p>
-</li>
-<li>
-<p>Add the carbonlib folder path in the Spark classpath. (Edit <code>$SPARK_HOME/conf/spark-env.sh</code> file and modify the value of <code>SPARK_CLASSPATH</code> by appending <code>$SPARK_HOME/carbonlib/*</code> to the existing value)</p>
-</li>
-<li>
-<p>Copy the <code>./conf/carbon.properties.template</code> file from CarbonData repository to <code>$SPARK_HOME/conf/</code> folder and rename the file to <code>carbon.properties</code>.</p>
-</li>
-<li>
-<p>Repeat Step 2 to Step 5 in all the nodes of the cluster.</p>
-</li>
-<li>
-<p>In Spark node[master], configure the properties mentioned in the following table in <code>$SPARK_HOME/conf/spark-defaults.conf</code> file.</p>
-</li>
-</ol>
-<table>
-<thead>
-<tr>
-<th>Property</th>
-<th>Value</th>
-<th>Description</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>spark.driver.extraJavaOptions</td>
-<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
-<td>A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.</td>
-</tr>
-<tr>
-<td>spark.executor.extraJavaOptions</td>
-<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
-<td>A string of extra JVM options to pass to executors. For instance, GC settings or other logging. <strong>NOTE</strong>: You can enter multiple values separated by space.</td>
-</tr>
-</tbody>
-</table>
-<ol start="7">
-<li>Add the following properties in <code>$SPARK_HOME/conf/carbon.properties</code> file:</li>
-</ol>
-<table>
-<thead>
-<tr>
-<th>Property</th>
-<th>Required</th>
-<th>Description</th>
-<th>Example</th>
-<th>Remark</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>carbon.storelocation</td>
-<td>NO</td>
-<td>Location where data CarbonData will create the store and write the data in its own format. If not specified then it takes spark.sql.warehouse.dir path.</td>
-<td>hdfs://HOSTNAME:PORT/Opt/CarbonStore</td>
-<td>Propose to set HDFS directory</td>
-</tr>
-</tbody>
-</table>
-<ol start="8">
-<li>Verify the installation. For example:</li>
-</ol>
-<pre><code>./spark-shell --master spark://HOSTNAME:PORT --total-executor-cores 2
---executor-memory 2G
-</code></pre>
-<p><strong>NOTE</strong>: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.</p>
-<p>To get started with CarbonData : <a href="quick-start-guide.html">Quick Start</a>, <a href="data-management-on-carbondata.html">Data Management on CarbonData</a></p>
-<h2>
-<a id="installing-and-configuring-carbondata-on-spark-on-yarn-cluster" class="anchor" href="#installing-and-configuring-carbondata-on-spark-on-yarn-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Spark on YARN Cluster</h2>
-<p>This section provides the procedure to install CarbonData on "Spark on YARN" cluster.</p>
-<h3>
-<a id="prerequisites-1" class="anchor" href="#prerequisites-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h3>
-<ul>
-<li>Hadoop HDFS and Yarn should be installed and running.</li>
-<li>Spark should be installed and running in all the clients.</li>
-<li>CarbonData user should have permission to access HDFS.</li>
-</ul>
-<h3>
-<a id="procedure-1" class="anchor" href="#procedure-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Procedure</h3>
-<p>The following steps are only for Driver Nodes. (Driver nodes are the one which starts the spark context.)</p>
-<ol>
-<li>
-<p><a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>Build the CarbonData</a> project and get the assembly jar from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code> and copy to <code>$SPARK_HOME/carbonlib</code> folder.</p>
-<p><strong>NOTE</strong>: Create the carbonlib folder if it does not exists inside <code>$SPARK_HOME</code> path.</p>
-</li>
-<li>
-<p>Copy the <code>./conf/carbon.properties.template</code> file from CarbonData repository to <code>$SPARK_HOME/conf/</code> folder and rename the file to <code>carbon.properties</code>.</p>
-</li>
-<li>
-<p>Create <code>tar.gz</code> file of carbonlib folder and move it inside the carbonlib folder.</p>
-</li>
-</ol>
-<pre><code>cd $SPARK_HOME
-tar -zcvf carbondata.tar.gz carbonlib/
-mv carbondata.tar.gz carbonlib/
-</code></pre>
-<ol start="4">
-<li>Configure the properties mentioned in the following table in <code>$SPARK_HOME/conf/spark-defaults.conf</code> file.</li>
-</ol>
-<table>
-<thead>
-<tr>
-<th>Property</th>
-<th>Description</th>
-<th>Value</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>spark.master</td>
-<td>Set this value to run the Spark in yarn cluster mode.</td>
-<td>Set yarn-client to run the Spark in yarn cluster mode.</td>
-</tr>
-<tr>
-<td>spark.yarn.dist.files</td>
-<td>Comma-separated list of files to be placed in the working directory of each executor.</td>
-<td><code>$SPARK_HOME/conf/carbon.properties</code></td>
-</tr>
-<tr>
-<td>spark.yarn.dist.archives</td>
-<td>Comma-separated list of archives to be extracted into the working directory of each executor.</td>
-<td><code>$SPARK_HOME/carbonlib/carbondata.tar.gz</code></td>
-</tr>
-<tr>
-<td>spark.executor.extraJavaOptions</td>
-<td>A string of extra JVM options to pass to executors. For instance  <strong>NOTE</strong>: You can enter multiple values separated by space.</td>
-<td><code>-Dcarbon.properties.filepath = carbon.properties</code></td>
-</tr>
-<tr>
-<td>spark.executor.extraClassPath</td>
-<td>Extra classpath entries to prepend to the classpath of executors. <strong>NOTE</strong>: If SPARK_CLASSPATH is defined in spark-env.sh, then comment it and append the values in below parameter spark.driver.extraClassPath</td>
-<td><code>carbondata.tar.gz/carbonlib/*</code></td>
-</tr>
-<tr>
-<td>spark.driver.extraClassPath</td>
-<td>Extra classpath entries to prepend to the classpath of the driver. <strong>NOTE</strong>: If SPARK_CLASSPATH is defined in spark-env.sh, then comment it and append the value in below parameter spark.driver.extraClassPath.</td>
-<td><code>$SPARK_HOME/carbonlib/*</code></td>
-</tr>
-<tr>
-<td>spark.driver.extraJavaOptions</td>
-<td>A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.</td>
-<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
-</tr>
-</tbody>
-</table>
-<ol start="5">
-<li>Add the following properties in <code>$SPARK_HOME/conf/carbon.properties</code>:</li>
-</ol>
-<table>
-<thead>
-<tr>
-<th>Property</th>
-<th>Required</th>
-<th>Description</th>
-<th>Example</th>
-<th>Default Value</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>carbon.storelocation</td>
-<td>NO</td>
-<td>Location where CarbonData will create the store and write the data in its own format. If not specified then it takes spark.sql.warehouse.dir path.</td>
-<td>hdfs://HOSTNAME:PORT/Opt/CarbonStore</td>
-<td>Propose to set HDFS directory</td>
-</tr>
-</tbody>
-</table>
-<ol start="6">
-<li>Verify the installation.</li>
-</ol>
-<pre><code> ./bin/spark-shell --master yarn-client --driver-memory 1g
- --executor-cores 2 --executor-memory 2G
-</code></pre>
-<p><strong>NOTE</strong>: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.</p>
-<p>Getting started with CarbonData : <a href="quick-start-guide.html">Quick Start</a>, <a href="data-management-on-carbondata.html">Data Management on CarbonData</a></p>
-<h2>
-<a id="query-execution-using-carbondata-thrift-server" class="anchor" href="#query-execution-using-carbondata-thrift-server" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Query Execution Using CarbonData Thrift Server</h2>
-<h3>
-<a id="starting-carbondata-thrift-server" class="anchor" href="#starting-carbondata-thrift-server" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Starting CarbonData Thrift Server.</h3>
-<p>a. cd <code>$SPARK_HOME</code></p>
-<p>b. Run the following command to start the CarbonData thrift server.</p>
-<pre><code>./bin/spark-submit
---class org.apache.carbondata.spark.thriftserver.CarbonThriftServer
-$SPARK_HOME/carbonlib/$CARBON_ASSEMBLY_JAR &lt;carbon_store_path&gt;
-</code></pre>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Example</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>CARBON_ASSEMBLY_JAR</td>
-<td>CarbonData assembly jar name present in the <code>$SPARK_HOME/carbonlib/</code> folder.</td>
-<td>carbondata_2.xx-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar</td>
-</tr>
-<tr>
-<td>carbon_store_path</td>
-<td>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. If not specified then it takes spark.sql.warehouse.dir path.</td>
-<td><code>hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store</code></td>
-</tr>
-</tbody>
-</table>
-<p><strong>NOTE</strong>: From Spark 1.6, by default the Thrift server runs in multi-session mode. Which means each JDBC/ODBC connection owns a copy of their own SQL configuration and temporary function registry. Cached tables are still shared though. If you prefer to run the Thrift server in single-session mode and share all SQL configuration and temporary function registry, please set option <code>spark.sql.hive.thriftServer.singleSession</code> to <code>true</code>. You may either add this option to <code>spark-defaults.conf</code>, or pass it to <code>spark-submit.sh</code> via <code>--conf</code>:</p>
-<pre><code>./bin/spark-submit
---conf spark.sql.hive.thriftServer.singleSession=true
---class org.apache.carbondata.spark.thriftserver.CarbonThriftServer
-$SPARK_HOME/carbonlib/$CARBON_ASSEMBLY_JAR &lt;carbon_store_path&gt;
-</code></pre>
-<p><strong>But</strong> in single-session mode, if one user changes the database from one connection, the database of the other connections will be changed too.</p>
-<p><strong>Examples</strong></p>
-<ul>
-<li>Start with default memory and executors.</li>
-</ul>
-<pre><code>./bin/spark-submit
---class org.apache.carbondata.spark.thriftserver.CarbonThriftServer 
-$SPARK_HOME/carbonlib
-/carbondata_2.xx-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar
-hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
-</code></pre>
-<ul>
-<li>Start with Fixed executors and resources.</li>
-</ul>
-<pre><code>./bin/spark-submit
---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://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
-</code></pre>
-<h3>
-<a id="connecting-to-carbondata-thrift-server-using-beeline" class="anchor" href="#connecting-to-carbondata-thrift-server-using-beeline" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Connecting to CarbonData Thrift Server Using Beeline.</h3>
-<pre><code>     cd $SPARK_HOME
-     ./sbin/start-thriftserver.sh
-     ./bin/beeline -u jdbc:hive2://&lt;thriftserver_host&gt;:port
-
-     Example
-     ./bin/beeline -u jdbc:hive2://10.10.10.10:10000
-</code></pre>
-</div>
-</div>
-</div>
-</div>
-<div class="doc-footer">
-    <a href="#top" class="scroll-top">Top</a>
-</div>
-</div>
-</section>
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/introduction.html
----------------------------------------------------------------------
diff --git a/content/introduction.html b/content/introduction.html
index 29d2aa1..068d711 100644
--- a/content/introduction.html
+++ b/content/introduction.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -228,122 +228,128 @@
 </li>
 </ul>
 <h2>
-<a id="architecture" class="anchor" href="#architecture" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Architecture</h2>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_architecture.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_architecture.png" alt="" style="max-width:100%;"></a></p>
-<h4>
-<a id="spark-interface-layer" class="anchor" href="#spark-interface-layer" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Spark Interface Layer:</h4>
-<p>CarbonData has deep integration with Apache Spark.CarbonData integrates custom Parser,Strategies,Optimization rules into Spark to take advantage of computing performed closer to data.</p>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_spark_integration.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_spark_integration.png" alt="" style="max-width:100%;"></a></p>
-<ol>
+<a id="carbondata-features--functions" class="anchor" href="#carbondata-features--functions" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Features &amp; Functions</h2>
+<p>CarbonData has rich set of featues to support various use cases in Big Data analytics.The below table lists the major features supported by CarbonData.</p>
+<h3>
+<a id="table-management" class="anchor" href="#table-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Table Management</h3>
+<ul>
 <li>
-<strong>Carbon parser</strong> Enhances Spark?s SQL parser to support Carbon specific DDL and DML commands to create carbon table, create aggregate tables, manage data loading, data retention and cleanup.</li>
+<h5>
+<a id="ddl-create-alterdropctas" class="anchor" href="#ddl-create-alterdropctas" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DDL (Create, Alter,Drop,CTAS)</h5>
+</li>
+</ul>
+<p>?	CarbonData provides its own DDL to create and manage carbondata tables.These DDL conform to 			Hive,Spark SQL format and support additional properties and configuration to take advantages of CarbonData functionalities.</p>
+<ul>
 <li>
-<strong>Carbon Strategies</strong>:- Modify Spark SQL?s physical query execution plan to push down possible operations to Carbon for example:- Grouping, Distinct Count, Top N etc.. for improving query performance.</li>
+<h5>
+<a id="dmlloadinsert" class="anchor" href="#dmlloadinsert" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DML(Load,Insert)</h5>
+<p>CarbonData provides its own DML to manage data in carbondata tables.It adds many customizations through configurations to completely customize the behavior as per user requirement scenarios.</p>
+</li>
 <li>
-<strong>Carbon Data RDD</strong>:- Makes the data present in Carbon tables visible to Spark as a RDD which enables spark to perform distributed computation on Carbon tables.</li>
-</ol>
-<h4>
-<a id="carbon-processor" class="anchor" href="#carbon-processor" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Carbon Processor:</h4>
-<p>Receives a query execution fragment from spark and executes the same on the Carbon storage. This involves Scanning the carbon store files for matching record, using the indices to directly locate the row sets and even the rows that may containing the data being searched for. The Carbon processor also performs all pushed down operations such as</p>
-<p>Aggregation/Group By</p>
-<p>Distinct Count</p>
-<p>Top N</p>
-<p>Expression Evaluation</p>
-<p>And many more?</p>
-<h4>
-<a id="carbon-storage" class="anchor" href="#carbon-storage" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Carbon Storage:</h4>
-<p>Custom columnar data store which is heavily compressed, binary, dictionary encoded and heavily indexed.Usaually stored in HDFS.</p>
-<h2>
-<a id="carbondata-features" class="anchor" href="#carbondata-features" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Features</h2>
-<p>CarbonData has rich set of featues to support various use cases in Big Data analytics.</p>
-<h2>
-<a id="design" class="anchor" href="#design" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Design</h2>
-<ul>
+<h5>
+<a id="update-and-delete" class="anchor" href="#update-and-delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Update and Delete</h5>
+<p>CarbonData supports Update and Delete on Big Data.CarbonData provides the syntax similar to Hive to support IUD operations on CarbonData tables.</p>
+</li>
 <li>
-<h3>
-<a id="dictionary-encoding" class="anchor" href="#dictionary-encoding" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Dictionary Encoding</h3>
+<h5>
+<a id="segment-management" class="anchor" href="#segment-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Segment Management</h5>
+<p>CarbonData has unique concept of segments to manage incremental loads to CarbonData tables effectively.Segment management helps to easily control the table, perform easy retention, and is also used to provide transaction capability for operations being performed.</p>
+</li>
+<li>
+<h5>
+<a id="partition" class="anchor" href="#partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Partition</h5>
+<p>CarbonData supports 2 kinds of partitions.1.partition similar to hive partition.2.CarbonData partition supporting hash,list,range partitioning.</p>
+</li>
+<li>
+<h5>
+<a id="compaction" class="anchor" href="#compaction" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Compaction</h5>
+<p>CarbonData manages incremental loads as segments.Compaction help to compact the growing number of segments and also to improve query filter pruning.</p>
+</li>
+<li>
+<h5>
+<a id="external-tables" class="anchor" href="#external-tables" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>External Tables</h5>
+<p>CarbonData can read any carbondata file and automatically infer schema from the file and provide a relational table view to perform sql queries using Spark or any other applicaion.</p>
 </li>
 </ul>
-<p>CarbonData supports encoding of data with suggogate values to reduce storage space and speed up processing.Most databases and big data SQL data stores adopt dictionary encoding(integer surrogate numbers) to achieve data compression.Unlike other column store databases where the dictionary is local to each data block, CarbonData maintains a global dictionary which provides opportunity for lazy conversion to actual values enabling all computation to be performed on the lightweight surrogate values.</p>
+<h3>
+<a id="datamaps" class="anchor" href="#datamaps" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DataMaps</h3>
+<ul>
+<li>
 <h5>
-<a id="dictionary-generation" class="anchor" href="#dictionary-generation" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Dictionary generation</h5>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_dict_encoding.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_dict_encoding.png" alt="" style="max-width:100%;"></a></p>
+<a id="pre-aggregate" class="anchor" href="#pre-aggregate" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Pre-Aggregate</h5>
+<p>CarbonData has concept of datamaps to assist in pruning of data while querying so that performance is faster.Pre Aggregate tables are kind of datamaps which can improve the query performance by order of magnitude.CarbonData will automatically pre-aggregae the incremental data and re-write the query to automatically fetch from the most appropriate pre-aggregate table to serve the query faster.</p>
+</li>
+<li>
 <h5>
-<a id=.htmlk-indexing" class="anchor" href=".htmlk-indexing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>MDK Indexing</h5>
-<p>All the surrogate keys are byte packed to generate an MDK (Multi Dimensional Key) Index.</p>
-<p>Any non surrogate columns of String data types are compressed using one of the configured compression algorithms and stored.For those numeric columns where surrogates are not generated, such data is stored as it is after compression.</p>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk.png" alt="image-20180903212418381" style="max-width:100%;"></a></p>
+<a id="time-series" class="anchor" href="#time-series" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Time Series</h5>
+<p>CarbonData has built in understanding of time order(Year, month,day,hour, minute,second).Time series is a pre-aggregate table which can automatically roll-up the data to the desired level during incremental load and serve the query from the most appropriate pre-aggregate table.</p>
+</li>
+<li>
+<h5>
+<a id="bloom-filter" class="anchor" href="#bloom-filter" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Bloom filter</h5>
+<p>CarbonData supports bloom filter as a datamap in order to quickly and efficiently prune the data for scanning and acheive faster query performance.</p>
+</li>
+<li>
 <h5>
-<a id="sorted.htmlk" class="anchor" href="#sorted.htmlk" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Sorted MDK</h5>
-<p>The data is sorted based on the MDK Index.Sorting helps for logical grouping of similar data and there by aids in faster look up during query.</p>
-<h4>
-<a id="" class="anchor" href="#" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk_sort.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk_sort.png" alt="image-20180903212525214" style="max-width:100%;"></a>
-</h4>
+<a id="lucene" class="anchor" href="#lucene" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Lucene</h5>
+<p>Lucene is popular for indexing text data which are long.CarbonData provides a lucene datamap so that text columns can be indexed using lucene and use the index result for efficient pruning of data to be retrieved during query.</p>
+</li>
+<li>
 <h5>
-<a id="custom-columnar-encoding" class="anchor" href="#custom-columnar-encoding" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Custom Columnar Encoding</h5>
-<p>The Sorted MDK Index is split into each column.Unlike other stores where the column is compressed and stored as it is, CarbonData sorts this column data so that Binary Search can be performed on individual column data based on the filter conditions.This aids in magnitude increase in query performance and also in better compression.Since the individual column's data gets sorted, it is necessary to maintain the row mapping with the sorted MDK Index data in order to retrieve data from other columns which are not participating in filter.This row mapping is termed as <strong>Inverted Index</strong> and is stored along with the column data.The below picture depicts the logical column view.User has the option to <strong>turn off</strong> Inverted Index for such columns where filters are never applied or is very rare.In such cases, scanning would be sequential, but can aid in reducing the storage size(occupied due to inverted index data).</p>
-<h4>
-<a id="-1" class="anchor" href="#-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_blocklet_view.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_blocklet_view.png" alt="" style="max-width:100%;"></a>
-</h4>
+<a id="mv-materialized-views" class="anchor" href="#mv-materialized-views" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>MV (Materialized Views)</h5>
+<p>MVs are kind of pre-aggregate tables which can support efficent query re-write and processing.CarbonData provides MV which can rewrite query to fetch from any table(including non-carbondata tables).Typical usecase is to store the aggregated data of a non-carbondata fact table into carbondata and use mv to rewrite the query to fetch from carbondata.</p>
+</li>
+</ul>
+<h3>
+<a id="streaming" class="anchor" href="#streaming" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Streaming</h3>
 <ul>
 <li>
+<h5>
+<a id="spark-streaming" class="anchor" href="#spark-streaming" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Spark Streaming</h5>
+<p>CarbonData supports streaming of data into carbondata in near-realtime and make it immediately available for query.CarbonData provides a DSL to create source and sink tables easily without the need for the user to write his application.</p>
+</li>
+</ul>
 <h3>
-<a id="carbondata-storage-format" class="anchor" href="#carbondata-storage-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Storage Format</h3>
-<p>CarbonData has a unique storage structure which aids in efficient storage and retrieval of data.Please refer to <a href="#./file-structure-of-carbondata.html">File Structure of CarbonData</a> for detailed information on the format.</p>
+<a id="sdk" class="anchor" href="#sdk" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SDK</h3>
+<ul>
+<li>
+<h5>
+<a id="carbondata-writer" class="anchor" href="#carbondata-writer" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData writer</h5>
+<p>CarbonData supports writing data from non-spark application using SDK.Users can use SDK to generate carbondata files from custom applications.Typical usecase is to write the streaming application plugged in to kafka and use carbondata as sink(target) table for storing.</p>
 </li>
 <li>
-<h3>
-<a id="indexing" class="anchor" href="#indexing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Indexing</h3>
-<p>CarbonData maintains multiple indexes at multiple levels to assist in efficient pruning of unwanted data from scan during query.Also CarbonData has support for plugging in external indexing solutions to speed up the query process.</p>
 <h5>
-<a id="min-max-indexing" class="anchor" href="#min-max-indexing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Min-Max Indexing</h5>
-<p>Storing data along with index significantly accelerates query performance and reduces the I/O scans and CPU resources in case of filters in the query. CarbonData index consists of multiple levels of indices, a processing framework can leverage this index to reduce the number of tasks it needs to schedule and process. It can also do skip scan in more fine grained units (called blocklet) in task side scanning instead of scanning the whole file.  <strong>CarbonData maintains Min-Max Index for all the columns.</strong></p>
-<p>CarbonData maintains a separate index file which contains the footer information for efficient IO reads.</p>
-<p>Using the Min-Max info in these index files, two levels of filtering can be achieved.</p>
-<p>Min-Max at the carbondata file level,to efficiently prune the files when the filter condition doesn't fall in the range.This information when maintained at the Spark Driver, will help to efficiently schedule the tasks for scanning</p>
-<p>Min-Max at the blocklet level, to efficiently prune the blocklets when the filter condition doesn't fall in the range.This information when maintained at the executor can significantly reduce the amount unnecessary data processed by the executor tasks.</p>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-minmax-blocklet.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-minmax-blocklet.png" alt="" style="max-width:100%;"></a></p>
+<a id="carbondata-reader" class="anchor" href="#carbondata-reader" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData reader</h5>
+<p>CarbonData supports reading of data from non-spark application using SDK.Users can use the SDK to read the carbondata files from their application and do custom processing.</p>
 </li>
+</ul>
+<h3>
+<a id="storage" class="anchor" href="#storage" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Storage</h3>
+<ul>
 <li>
-<h4>
-<a id="datamaps" class="anchor" href="#datamaps" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DataMaps</h4>
-<p>DataMap is a framework for indexing and also for statistics that can be used to add primary index (Blocklet Index) , secondary index type and statistical type to CarbonData.</p>
-<p>DataMap is a standardized general interface which CarbonData uses to prune data blocks for scanning.</p>
-<p>DataMaps are of 2 types:</p>
-<p><strong>CG(Coarse Grained) DataMaps</strong> Can prune data to the blocklet or to Page level.ie., Holds information for deciding which blocks/blocklets to be scanned.This DataMap is used in Spark Driver to decide the number of tasks to be scheduled.</p>
-<p><strong>FG(Fine Grained) DataMaps</strong> Can prune data to row level.This DataMap is used in Spark executor for scanning an fetching the data much faster.</p>
-<p>Since DataMap interfaces are generalised, We can write a thin adaptor called as <strong>DataMap Providers</strong> to interface between CarbonData and other external Indexing engines. For eg., Lucene, Solr,ES,...</p>
-<p>CarbonData has its own DSL to create and manage DataMaps.Please refer to <a href="#./datamap/datamap-management.html#overview">CarbonData DSL</a> for more information.</p>
-<p>The below diagram explains about the DataMap execution in CarbonData.</p>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-datamap.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-datamap.png" alt="" style="max-width:100%;"></a></p>
+<h5>
+<a id="s3" class="anchor" href="#s3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>S3</h5>
+<p>CarbonData can write to S3, OBS or any cloud storage confirming to S3 protocol.CarbonData uses the HDFS api to write to cloud object stores.</p>
 </li>
 <li>
-<h4>
-<a id="update--delete" class="anchor" href="#update--delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Update &amp; Delete</h4>
+<h5>
+<a id="hdfs" class="anchor" href="#hdfs" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>HDFS</h5>
+<p>CarbonData uses HDFS api to write and read data from HDFS.CarbonData can take advantage of the locality information to efficiently suggest spark to run tasks near to the data.</p>
 </li>
 </ul>
-<p>CarbonData supports Update and delete operations over big data.This functionality is not targetted for OLTP scenarios where high concurrent update/delete are required.Following are the assumptions considered when this feature is designed.</p>
-<ol>
-<li>Updates or Deletes are periodic and in Bulk</li>
-<li>Updates or Deletes are atomic</li>
-<li>Data is immediately visible</li>
-<li>Concurrent query to be allowed during an update or delete operation</li>
-<li>Single statement auto-commit support (not OLTP-style transaction)</li>
-</ol>
-<p>Since data stored in HDFS are immutable,data blocks cannot be updated in-place.Re-write of entire data block is not efficient for IO and also is a slow process.</p>
-<p>To over come these limitations, CarbonData adopts methodology of writing a delta file containing the rows to be deleted and another delta file containing the values to be updated with.During processing, These two delta files are merged with the main carbondata file and the correct result is returned for the query.</p>
-<p>The below diagram describes the process.</p>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_update_delete.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_update_delete.png" alt="" style="max-width:100%;"></a></p>
 <h2>
 <a id="integration-with-big-data-ecosystem" class="anchor" href="#integration-with-big-data-ecosystem" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Integration with Big Data ecosystem</h2>
-<p>Refer to Integration with <a href="#./quick-start-guide.html#spark">Spark</a>, <a href="#./quick-start-guide.html#presto">Presto</a> for detailed information on integrating CarbonData with these execution engines.</p>
+<p>Refer to Integration with <a href="./quick-start-guide.html#spark">Spark</a>, <a href="./quick-start-guide.html#presto">Presto</a> for detailed information on integrating CarbonData with these execution engines.</p>
 <h2>
 <a id="scenarios-where-carbondata-is-suitable" class="anchor" href="#scenarios-where-carbondata-is-suitable" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Scenarios where CarbonData is suitable</h2>
+<p>CarbonData is useful in various analytical work loads.Some of the most typical usecases where CarbonData is being used is <a href="./usecases.html">documented here</a>.</p>
+<h2>
+<a id="performance-results" class="anchor" href="#performance-results" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Performance Results</h2>
+<p><a href="../docs/images/carbondata-performance.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbondata-performance.png?raw=true" alt="Performance Results" style="max-width:100%;"></a></p>
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__intro').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -358,4 +364,4 @@ $(function() { $('.b-nav__intro').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/language-manual.html
----------------------------------------------------------------------
diff --git a/content/language-manual.html b/content/language-manual.html
index 54c29ac..a0ea674 100644
--- a/content/language-manual.html
+++ b/content/language-manual.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -220,7 +220,6 @@
 <li>
 <a href="./ddl-of-carbondata.html">DDL:</a><a href="./ddl-of-carbondata.html#create-table">Create</a>,<a href="./ddl-of-carbondata.html#drop-table">Drop</a>,<a href="./ddl-of-carbondata.html#partition">Partition</a>,<a href="./ddl-of-carbondata.html#bucketing">Bucketing</a>,<a href="./ddl-of-carbondata.html#alter-table">Alter</a>,<a href="./ddl-of-carbondata.html#create-table-as-select">CTAS</a>,<a href="./ddl-of-carbondata.html#create-external-table">External Table</a>
 </li>
-<li>Indexes</li>
 <li>
 <a href="./datamap-management.html">DataMaps</a>
 <ul>
@@ -254,8 +253,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -270,4 +268,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/lucene-datamap-guide.html
----------------------------------------------------------------------
diff --git a/content/lucene-datamap-guide.html b/content/lucene-datamap-guide.html
index 038cb9d..b8164a2 100644
--- a/content/lucene-datamap-guide.html
+++ b/content/lucene-datamap-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -248,7 +248,7 @@ search tokenized word or pattern of it using lucene query on text content.</p>
   age int,
   city string,
   country string)
-STORED BY 'carbondata'
+STORED AS carbondata
 </code></pre>
 <p>User can create Lucene datamap using the Create DataMap DDL:</p>
 <pre><code>CREATE DATAMAP dm
@@ -328,7 +328,7 @@ select * from datamap_test where TEXT_MATCH('name:*n*')
 
 select * from datamap_test where TEXT_MATCH('name:*10 -name:*n*')
 </code></pre>
-<p><strong>Note:</strong> For lucene queries and syntax, refer to <a href="www.lucenetutorial.com/lucene-query-syntax.html">lucene-syntax</a></p>
+<p><strong>Note:</strong> For lucene queries and syntax, refer to <a href="http://www.lucenetutorial.com/lucene-query-syntax.html" target=_blank rel="nofollow">lucene-syntax</a></p>
 <h2>
 <a id="data-management-with-lucene-datamap" class="anchor" href="#data-management-with-lucene-datamap" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Management with lucene datamap</h2>
 <p>Once there is lucene datamap is created on the main table, following command on the main
@@ -363,8 +363,7 @@ $(function() {
     $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -379,4 +378,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/mainpage.html
----------------------------------------------------------------------
diff --git a/content/mainpage.html b/content/mainpage.html
deleted file mode 100644
index d515853..0000000
--- a/content/mainpage.html
+++ /dev/null
@@ -1,214 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
-                                   target="_blank">Apache CarbonData 1.4.1</a></li>
-							<li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
-                                   target="_blank">Apache CarbonData 1.4.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
-                                   target="_blank">Apache CarbonData 1.3.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
-                                   target="_blank">Apache CarbonData 1.3.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input" placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="doc-heading">
-                                <h4 class="title">Documentation
-                                    <span class="title-underline"></span>
-                                </h4>
-                            </div>
-
-                            <div class="row">
-
-                                <div class="col-sm-12  col-md-12">
-                                    <span class="text-justify">
-                                        Welcome to Apache CarbonData. Apache CarbonData is a new big data file format for faster interactive query using advanced columnar storage, index, compression and encoding techniques to improve computing efficiency, which helps in speeding up queries by an order of magnitude faster over PetaBytes of data. This user guide provides a detailed description about the CarbonData and its features.
-                                        Let's get started !
-                                    </span>
-                                    <hr style="margin: 12px 0 8px">
-                                    <div>
-                                        <ul class="sub-nav">
-                                            <li><a href="quick-start-guide.html">Quick Start</a></li>
-                                            <li><a href="file-structure-of-carbondata.html">CarbonData File Structure</a></li>
-                                            <li><a href="supported-data-types-in-carbondata.html">Data Types</a></li>
-                                            <li><a href="data-management-on-carbondata.html">Data Management On CarbonData</a></li>
-                                            <li><a href="installation-guide.html">Installation Guide</a></li>
-                                            <li><a href="configuration-parameters.html">Configuring CarbonData</a></li>
-                                            <li><a href="streaming-guide.html">Streaming Guide</a></li>
-                                            <li><a href="sdk-guide.html">SDK Guide</a></li>
-											<li><a href="s3-guide.html">S3 Guide (Alpha Feature)</a></li>
-                                            <li><a href="datamap-developer-guide.html">DataMap Developer Guide</a></li>
-											<li><a href="datamap-management.html">CarbonData DataMap Management</a></li>
-                                            <li><a href="bloomfilter-datamap-guide.html">CarbonData BloomFilter DataMap (Alpha Feature)</a></li>
-                                            <li><a href="lucene-datamap-guide.html">CarbonData Lucene DataMap (Alpha Feature)</a></li>
-                                            <li><a href="preaggregate-datamap-guide.html">CarbonData Pre-aggregate DataMap</a></li>
-                                            <li><a href="timeseries-datamap-guide.html">CarbonData Timeseries DataMap</a></li>
-                                            <li><a href="faq.html">FAQs</a></li>
-                                            <li><a href="troubleshooting.html">Troubleshooting</a></li>
-                                            <li><a href="useful-tips-on-carbondata.html">Useful Tips</a></li>
-
-                                        </ul>
-                                    </div>
-                                </div>
-                            </div>
-                        </div>
-                        <div class="doc-footer">
-                            <a href="#top" class="scroll-top">Top</a>
-                        </div>
-                    </div>
-                </section>
-            </div>
-        </div>
-    </div>
-</section><!-- End systemblock part -->
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/partition-guide.html
----------------------------------------------------------------------
diff --git a/content/partition-guide.html b/content/partition-guide.html
index 4e6e6c0..a9a1f8c 100644
--- a/content/partition-guide.html
+++ b/content/partition-guide.html
@@ -339,4 +339,4 @@ SegmentDir/part-0-0_batchno0-0-1502703086921.carbondata
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/performance-tuning.html
----------------------------------------------------------------------
diff --git a/content/performance-tuning.html b/content/performance-tuning.html
index 49b3d3a..480911c 100644
--- a/content/performance-tuning.html
+++ b/content/performance-tuning.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -218,6 +218,7 @@ The following sections will elaborate on the below topics :</p>
 <li><a href="#suggestions-to-create-carbondata-table">Suggestions to create CarbonData Table</a></li>
 <li><a href="#configuration-for-optimizing-data-loading-performance-for-massive-data">Configuration for Optimizing Data Loading performance for Massive Data</a></li>
 <li><a href="#configurations-for-optimizing-carbondata-performance">Optimizing Query Performance</a></li>
+<li><a href="#compaction-configurations-for-optimizing-carbondata-query-performance">Compaction Configurations for Optimizing CarbonData Query Performance</a></li>
 </ul>
 <h2>
 <a id="suggestions-to-create-carbondata-table" class="anchor" href="#suggestions-to-create-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Suggestions to Create CarbonData Table</h2>
@@ -299,7 +300,7 @@ The create table command can be modified as suggested below :</p>
   counter_1, Decimal
   ...
   
-  )STORED BY 'carbondata'
+  )STORED AS carbondata
   TBLPROPERTIES ('SORT_COLUMNS'='msisdn, Dime_1')
 </code></pre>
 <p>Now the query with MSISDN in the filter will be more efficient.</p>
@@ -320,7 +321,7 @@ The create table command can be modified as suggested below :</p>
     counter_1, Decimal
     ...
     
-    )STORED BY 'carbondata'
+    )STORED AS carbondata
     TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
 </code></pre>
 <ul>
@@ -337,7 +338,7 @@ The create table command can be modified as below :</p>
     counter_1 decimal,
     counter_2 double,
     ...
-    )STORED BY 'carbondata'
+    )STORED AS carbondata
     TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
 </code></pre>
 <p>The result of performance analysis of test-case shows reduction in query execution time from 15 to 3 seconds, thereby improving performance by nearly 5 times.</p>
@@ -356,12 +357,12 @@ Incremental values are efficient in using min/max index. The create table comman
   END_TIME bigint,
   ...
   counter_100 double
-  )STORED BY 'carbondata'
+  )STORED AS carbondata
   TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
 </code></pre>
 <p><strong>NOTE:</strong></p>
 <ul>
-<li>BloomFilter can be created to enhance performance for queries with precise equal/in conditions. You can find more information about it in BloomFilter datamap <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/bloomfilter-datamap-guide.html" target=_blank>document</a>.</li>
+<li>BloomFilter can be created to enhance performance for queries with precise equal/in conditions. You can find more information about it in BloomFilter datamap <a href="./bloomfilter-datamap-guide.html">document</a>.</li>
 </ul>
 <h2>
 <a id="configuration-for-optimizing-data-loading-performance-for-massive-data" class="anchor" href="#configuration-for-optimizing-data-loading-performance-for-massive-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configuration for Optimizing Data Loading performance for Massive Data</h2>
@@ -507,11 +508,72 @@ scenarios. After the completion of POC, some of the configurations impacting the
 </tbody>
 </table>
 <p>Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.</p>
+<h2>
+<a id="compaction-configurations-for-optimizing-carbondata-query-performance" class="anchor" href="#compaction-configurations-for-optimizing-carbondata-query-performance" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Compaction Configurations for Optimizing CarbonData Query Performance</h2>
+<p>CarbonData provides many configurations to tune the compaction behavior so that query peformance is improved.</p>
+<p>Based on the number of cores available in the node, it is recommended to tune the configuration 	<em><strong>carbon.number.of.cores.while.compacting</strong></em> appropriately.Configuring a higher value will improve the overall compaction performance.</p>
+<p> </p>
+<table>
+<tbody>
+<tr>
+<td>No</td>
+<td> Data Loading frequency</td>
+<td>Data Size of each load</td>
+<td>Minor Compaction configuration</td>
+<td> Major compaction configuration</td>
+</tr>
+<tr>
+<td>1</td>
+<td> Batch(Once is several Hours)</td>
+<td>Big</td>
+<td> Not Suggested</td>
+<td>Configure Major Compaction size of 3-4 load size.Perform Major compaction once in a day</td>
+</tr>
+<tr>
+<td rowspan="2">2</td>
+<td rowspan="2"> Batch(Once in few minutes) </td>
+<td>Big </td>
+<td>
+<p> Minor compaction (2,2).</p>
+<p>Enable Auto compaction, if high rate data loading speed is not required or the time between loads is sufficient to run the compaction</p>
+</td>
+<td>Major compaction size of 10 load size.Perform Major compaction once in a day</td>
+</tr>
+<tr>
+<td>Small</td>
+<td>
+<p>Minor compaction (6,6).</p>
+<p>Enable Auto compaction, if high rate data loading speed is not required or the time between loads is sufficient to run the compaction</p>
+</td>
+<td>Major compaction size of 10 load size.Perform Major compaction once in a day</td>
+</tr>
+<tr>
+<td>3</td>
+<td> History data loaded as single load,incremental loads matches (1) or (2)</td>
+<td>Big</td>
+<td>
+<p> Configure ALLOWED_COMPACTION_DAYS to exclude the History load.</p>
+<p>Configure Minor compaction configuration based condition (1) or (2)</p>
+</td>
+<td> Configure Major compaction size smaller than the history load size.</td>
+</tr>
+<tr>
+<td>4</td>
+<td> There can be error in recent data loaded.Need reload sometimes</td>
+<td> (1) or (2)</td>
+<td>
+<p> Configure COMPACTION_PRESERVE_SEGMENTS</p>
+<p>to exclude the recent few segments from compacting.</p>
+<p>Configure Minor compaction configuration based condition (1) or (2)</p>
+</td>
+<td>Same as (1) or (2) </td>
+</tr>
+</tbody>
+</table>
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__perf').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -526,4 +588,4 @@ $(function() { $('.b-nav__perf').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/preaggregate-datamap-guide.html
----------------------------------------------------------------------
diff --git a/content/preaggregate-datamap-guide.html b/content/preaggregate-datamap-guide.html
index 9220c84..6b0783e 100644
--- a/content/preaggregate-datamap-guide.html
+++ b/content/preaggregate-datamap-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -256,7 +256,7 @@
 <span class="pl-s">      | country string,</span>
 <span class="pl-s">      | quantity int,</span>
 <span class="pl-s">      | price bigint)</span>
-<span class="pl-s">      | STORED BY 'carbondata'</span>
+<span class="pl-s">      | STORED AS carbondata</span>
 <span class="pl-s">    <span class="pl-pds">"""</span></span>.stripMargin)
  
  <span class="pl-c"><span class="pl-c">//</span> Create pre-aggregate table on the main table</span>
@@ -347,7 +347,7 @@ in production SQLs.</p>
   country string,
   quantity int,
   price bigint)
-STORED BY 'carbondata'
+STORED AS carbondata
 </code></pre>
 <p>User can create pre-aggregate tables using the Create DataMap DDL</p>
 <pre><code>CREATE DATAMAP agg_sales
@@ -476,8 +476,7 @@ $(function() {
     $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -492,4 +491,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file


[02/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/installation-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/installation-guide.md b/src/site/markdown/installation-guide.md
deleted file mode 100644
index f679338..0000000
--- a/src/site/markdown/installation-guide.md
+++ /dev/null
@@ -1,198 +0,0 @@
-<!--
-    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. If not specified then it takes spark.sql.warehouse.dir path. | 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), [Data Management on CarbonData](data-management-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. If not specified then it takes spark.sql.warehouse.dir path.| 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), [Data Management on CarbonData](data-management-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
---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. If not specified then it takes spark.sql.warehouse.dir path. | `hdfs://<host_name>:port/user/hive/warehouse/carbon.store` |
-
-**NOTE**: From Spark 1.6, by default the Thrift server runs in multi-session mode. Which means each JDBC/ODBC connection owns a copy of their own SQL configuration and temporary function registry. Cached tables are still shared though. If you prefer to run the Thrift server in single-session mode and share all SQL configuration and temporary function registry, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add this option to `spark-defaults.conf`, or pass it to `spark-submit.sh` via `--conf`:
-
-```
-./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>
-```
-
-**But** in single-session mode, if one user changes the database from one connection, the database of the other connections will be changed too.
-
-**Examples**
-   
-   * Start with default memory and executors.
-
-```
-./bin/spark-submit
---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
---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/44eed099/src/site/markdown/introduction.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/introduction.md b/src/site/markdown/introduction.md
new file mode 100644
index 0000000..8169958
--- /dev/null
+++ b/src/site/markdown/introduction.md
@@ -0,0 +1,172 @@
+## What is CarbonData
+
+CarbonData is a fully indexed columnar and Hadoop native data-store for processing heavy analytical workloads and detailed queries on big data with Spark SQL. CarbonData allows faster interactive queries over PetaBytes of data.
+
+
+
+## What does this mean
+
+CarbonData has specially engineered optimizations like multi level indexing, compression and encoding techniques targeted to improve performance of analytical queries which can include filters, aggregation and distinct counts where users expect sub second response time for queries on TB level data on commodity hardware clusters with just a few nodes.
+
+CarbonData has 
+
+- **Unique data organisation** for faster retrievals and minimise amount of data retrieved
+
+- **Advanced push down optimisations** for deep integration with Spark so as to improvise the Spark DataSource API and other experimental features thereby ensure computing is performed close to the data to minimise amount of data read, processed, converted and transmitted(shuffled) 
+
+- **Multi level indexing** to efficiently prune the files and data to be scanned and hence reduce I/O scans and CPU processing
+
+
+
+## Architecture
+
+![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_architecture.png)
+
+
+
+#### Spark Interface Layer: 
+
+CarbonData has deep integration with Apache Spark.CarbonData integrates custom Parser,Strategies,Optimization rules into Spark to take advantage of computing performed closer to data.
+
+![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_spark_integration.png)
+
+1. **Carbon parser** Enhances Spark’s SQL parser to support Carbon specific DDL and DML commands to create carbon table, create aggregate tables, manage data loading, data retention and cleanup.
+2. **Carbon Strategies**:- Modify Spark SQL’s physical query execution plan to push down possible operations to Carbon for example:- Grouping, Distinct Count, Top N etc.. for improving query performance.
+3. **Carbon Data RDD**:- Makes the data present in Carbon tables visible to Spark as a RDD which enables spark to perform distributed computation on Carbon tables.
+
+
+
+#### Carbon Processor: 
+
+Receives a query execution fragment from spark and executes the same on the Carbon storage. This involves Scanning the carbon store files for matching record, using the indices to directly locate the row sets and even the rows that may containing the data being searched for. The Carbon processor also performs all pushed down operations such as 
+
+Aggregation/Group By
+
+Distinct Count
+
+Top N
+
+Expression Evaluation
+
+And many more…
+
+#### Carbon Storage:
+
+Custom columnar data store which is heavily compressed, binary, dictionary encoded and heavily indexed.Usaually stored in HDFS.
+
+## CarbonData Features
+
+CarbonData has rich set of featues to support various use cases in Big Data analytics.
+
+ 
+
+## Design
+
+- ### Dictionary Encoding
+
+CarbonData supports encoding of data with suggogate values to reduce storage space and speed up processing.Most databases and big data SQL data stores adopt dictionary encoding(integer surrogate numbers) to achieve data compression.Unlike other column store databases where the dictionary is local to each data block, CarbonData maintains a global dictionary which provides opportunity for lazy conversion to actual values enabling all computation to be performed on the lightweight surrogate values.
+
+##### Dictionary generation
+
+![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_dict_encoding.png)
+
+
+
+##### MDK Indexing
+
+All the surrogate keys are byte packed to generate an MDK (Multi Dimensional Key) Index.
+
+Any non surrogate columns of String data types are compressed using one of the configured compression algorithms and stored.For those numeric columns where surrogates are not generated, such data is stored as it is after compression.
+
+![image-20180903212418381](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_mdk.png)
+
+##### Sorted MDK
+
+The data is sorted based on the MDK Index.Sorting helps for logical grouping of similar data and there by aids in faster look up during query.
+
+#### ![image-20180903212525214](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_mdk_sort.png)
+
+##### Custom Columnar Encoding
+
+The Sorted MDK Index is split into each column.Unlike other stores where the column is compressed and stored as it is, CarbonData sorts this column data so that Binary Search can be performed on individual column data based on the filter conditions.This aids in magnitude increase in query performance and also in better compression.Since the individual column's data gets sorted, it is necessary to maintain the row mapping with the sorted MDK Index data in order to retrieve data from other columns which are not participating in filter.This row mapping is termed as **Inverted Index** and is stored along with the column data.The below picture depicts the logical column view.User has the option to **turn off** Inverted Index for such columns where filters are never applied or is very rare.In such cases, scanning would be sequential, but can aid in reducing the storage size(occupied due to inverted index data).
+
+#### ![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_blocklet_view.png)
+
+- ### CarbonData Storage Format
+
+  CarbonData has a unique storage structure which aids in efficient storage and retrieval of data.Please refer to [File Structure of CarbonData](#./file-structure-of-carbondata.md) for detailed information on the format.
+
+- ### Indexing
+
+  CarbonData maintains multiple indexes at multiple levels to assist in efficient pruning of unwanted data from scan during query.Also CarbonData has support for plugging in external indexing solutions to speed up the query process.
+
+  ##### Min-Max Indexing
+
+  Storing data along with index significantly accelerates query performance and reduces the I/O scans and CPU resources in case of filters in the query. CarbonData index consists of multiple levels of indices, a processing framework can leverage this index to reduce the number of tasks it needs to schedule and process. It can also do skip scan in more fine grained units (called blocklet) in task side scanning instead of scanning the whole file.  **CarbonData maintains Min-Max Index for all the columns.**
+
+  CarbonData maintains a separate index file which contains the footer information for efficient IO reads.
+
+  Using the Min-Max info in these index files, two levels of filtering can be achieved.
+
+  Min-Max at the carbondata file level,to efficiently prune the files when the filter condition doesn't fall in the range.This information when maintained at the Spark Driver, will help to efficiently schedule the tasks for scanning
+
+  Min-Max at the blocklet level, to efficiently prune the blocklets when the filter condition doesn't fall in the range.This information when maintained at the executor can significantly reduce the amount unnecessary data processed by the executor tasks. 
+
+
+
+  ![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-minmax-blocklet.png)
+
+- #### DataMaps
+
+  DataMap is a framework for indexing and also for statistics that can be used to add primary index (Blocklet Index) , secondary index type and statistical type to CarbonData.
+
+  DataMap is a standardized general interface which CarbonData uses to prune data blocks for scanning.
+
+  DataMaps are of 2 types:
+
+  **CG(Coarse Grained) DataMaps** Can prune data to the blocklet or to Page level.ie., Holds information for deciding which blocks/blocklets to be scanned.This DataMap is used in Spark Driver to decide the number of tasks to be scheduled.
+
+  **FG(Fine Grained) DataMaps** Can prune data to row level.This DataMap is used in Spark executor for scanning an fetching the data much faster.
+
+  Since DataMap interfaces are generalised, We can write a thin adaptor called as **DataMap Providers** to interface between CarbonData and other external Indexing engines. For eg., Lucene, Solr,ES,...
+
+  CarbonData has its own DSL to create and manage DataMaps.Please refer to [CarbonData DSL](#./datamap/datamap-management.md#overview) for more information.
+
+  The below diagram explains about the DataMap execution in CarbonData.
+
+  ![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-datamap.png)
+
+- #### Update & Delete
+
+
+CarbonData supports Update and delete operations over big data.This functionality is not targetted for OLTP scenarios where high concurrent update/delete are required.Following are the assumptions considered when this feature is designed.
+
+1. Updates or Deletes are periodic and in Bulk
+2. Updates or Deletes are atomic
+3. Data is immediately visible
+4. Concurrent query to be allowed during an update or delete operation
+5. Single statement auto-commit support (not OLTP-style transaction)
+
+Since data stored in HDFS are immutable,data blocks cannot be updated in-place.Re-write of entire data block is not efficient for IO and also is a slow process.
+
+To over come these limitations, CarbonData adopts methodology of writing a delta file containing the rows to be deleted and another delta file containing the values to be updated with.During processing, These two delta files are merged with the main carbondata file and the correct result is returned for the query.
+
+The below diagram describes the process.
+
+![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_update_delete.png)
+
+
+
+## Integration with Big Data ecosystem
+
+Refer to Integration with [Spark](#./quick-start-guide.md#spark), [Presto](#./quick-start-guide.md#presto) for detailed information on integrating CarbonData with these execution engines.
+
+## Scenarios where CarbonData is suitable
+
+
+
+
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__intro').addClass('selected'); });
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/language-manual.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/language-manual.md b/src/site/markdown/language-manual.md
new file mode 100644
index 0000000..9fef71b
--- /dev/null
+++ b/src/site/markdown/language-manual.md
@@ -0,0 +1,51 @@
+<!--
+    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.
+-->
+
+# Overview
+
+
+
+CarbonData has its own parser, in addition to Spark's SQL Parser, to parse and process certain Commands related to CarbonData table handling. You can interact with the SQL interface using the [command-line](https://spark.apache.org/docs/latest/sql-programming-guide.html#running-the-spark-sql-cli) or over [JDBC/ODBC](https://spark.apache.org/docs/latest/sql-programming-guide.html#running-the-thrift-jdbcodbc-server).
+
+- [Data Types](./supported-data-types-in-carbondata.md)
+- Data Definition Statements
+  - [DDL:](./ddl-of-carbondata.md)[Create](./ddl-of-carbondata.md#create-table),[Drop](./ddl-of-carbondata.md#drop-table),[Partition](./ddl-of-carbondata.md#partition),[Bucketing](./ddl-of-carbondata.md#bucketing),[Alter](./ddl-of-carbondata.md#alter-table),[CTAS](./ddl-of-carbondata.md#create-table-as-select),[External Table](./ddl-of-carbondata.md#create-external-table)
+  - Indexes
+  - [DataMaps](./datamap-management.md)
+    - [Bloom](./bloomfilter-datamap-guide.md)
+    - [Lucene](./lucene-datamap-guide.md)
+    - [Pre-Aggregate](./preaggregate-datamap-guide.md)
+    - [Time Series](./timeseries-datamap-guide.md)
+  - Materialized Views (MV)
+  - [Streaming](./streaming-guide.md)
+- Data Manipulation Statements
+  - [DML:](./dml-of-carbondata.md) [Load](./dml-of-carbondata.md#load-data), [Insert](./ddl-of-carbondata.md#insert-overwrite), [Update](./dml-of-carbondata.md#update), [Delete](./dml-of-carbondata.md#delete)
+  - [Segment Management](./segment-management-on-carbondata.md)
+- [Configuration Properties](./configuration-parameters.md)
+
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/lucene-datamap-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/lucene-datamap-guide.md b/src/site/markdown/lucene-datamap-guide.md
index 06cd194..248c8e5 100644
--- a/src/site/markdown/lucene-datamap-guide.md
+++ b/src/site/markdown/lucene-datamap-guide.md
@@ -173,4 +173,15 @@ release, user can do as following:
 3. Create the lucene datamap again by `CREATE DATAMAP` command.
 Basically, user can manually trigger the operation by re-building the datamap.
 
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/performance-tuning.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/performance-tuning.md b/src/site/markdown/performance-tuning.md
new file mode 100644
index 0000000..d8b53f2
--- /dev/null
+++ b/src/site/markdown/performance-tuning.md
@@ -0,0 +1,183 @@
+<!--
+    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.
+-->
+
+# Useful Tips
+  This tutorial guides you to create CarbonData Tables and optimize performance.
+  The following sections will elaborate on the below topics :
+
+  * [Suggestions to create CarbonData Table](#suggestions-to-create-carbondata-table)
+  * [Configuration for Optimizing Data Loading performance for Massive Data](#configuration-for-optimizing-data-loading-performance-for-massive-data)
+  * [Optimizing Query Performance](#configurations-for-optimizing-carbondata-performance)
+
+## Suggestions to Create CarbonData Table
+
+  For example, the results of the analysis for table creation with dimensions ranging from 10 thousand to 10 billion rows and 100 to 300 columns have been summarized below.
+  The following table describes some of the columns from the table used.
+
+  - **Table Column Description**
+
+| Column Name | Data Type     | Cardinality | Attribution |
+|-------------|---------------|-------------|-------------|
+| msisdn      | String        | 30 million  | Dimension   |
+| BEGIN_TIME  | BigInt        | 10 Thousand | Dimension   |
+| HOST        | String        | 1 million   | Dimension   |
+| Dime_1      | String        | 1 Thousand  | Dimension   |
+| counter_1   | Decimal       | NA          | Measure     |
+| counter_2   | Numeric(20,0) | NA          | Measure     |
+| ...         | ...           | NA          | Measure     |
+| counter_100 | Decimal       | NA          | Measure     |
+
+
+  - **Put the frequently-used column filter in the beginning of SORT_COLUMNS**
+
+  For example, MSISDN filter is used in most of the query then we must put the MSISDN as the first column in SORT_COLUMNS property.
+  The create table command can be modified as suggested below :
+
+  ```
+  create table carbondata_table(
+    msisdn String,
+    BEGIN_TIME bigint,
+    HOST String,
+    Dime_1 String,
+    counter_1, Decimal
+    ...
+    
+    )STORED BY 'carbondata'
+    TBLPROPERTIES ('SORT_COLUMNS'='msisdn, Dime_1')
+  ```
+
+  Now the query with MSISDN in the filter will be more efficient.
+
+  - **Put the frequently-used columns in the order of low to high cardinality in SORT_COLUMNS**
+
+  If the table in the specified query has multiple columns which are frequently used to filter the results, it is suggested to put
+  the columns in the order of cardinality low to high in SORT_COLUMNS configuration. This ordering of frequently used columns improves the compression ratio and
+  enhances the performance of queries with filter on these columns.
+
+  For example, if MSISDN, HOST and Dime_1 are frequently-used columns, then the column order of table is suggested as
+  Dime_1>HOST>MSISDN, because Dime_1 has the lowest cardinality.
+  The create table command can be modified as suggested below :
+
+  ```
+  create table carbondata_table(
+      msisdn String,
+      BEGIN_TIME bigint,
+      HOST String,
+      Dime_1 String,
+      counter_1, Decimal
+      ...
+      
+      )STORED BY 'carbondata'
+      TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
+  ```
+
+  - **For measure type columns with non high accuracy, replace Numeric(20,0) data type with Double data type**
+
+  For columns of measure type, not requiring high accuracy, it is suggested to replace Numeric data type with Double to enhance query performance. 
+  The create table command can be modified as below :
+
+```
+  create table carbondata_table(
+    Dime_1 String,
+    BEGIN_TIME bigint,
+    END_TIME bigint,
+    HOST String,
+    MSISDN String,
+    counter_1 decimal,
+    counter_2 double,
+    ...
+    )STORED BY 'carbondata'
+    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
+```
+  The result of performance analysis of test-case shows reduction in query execution time from 15 to 3 seconds, thereby improving performance by nearly 5 times.
+
+ - **Columns of incremental character should be re-arranged at the end of dimensions**
+
+  Consider the following scenario where data is loaded each day and the begin_time is incremental for each load, it is suggested to put begin_time at the end of dimensions.
+  Incremental values are efficient in using min/max index. The create table command can be modified as below :
+
+  ```
+  create table carbondata_table(
+    Dime_1 String,
+    HOST String,
+    MSISDN String,
+    counter_1 double,
+    counter_2 double,
+    BEGIN_TIME bigint,
+    END_TIME bigint,
+    ...
+    counter_100 double
+    )STORED BY 'carbondata'
+    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
+  ```
+
+  **NOTE:**
+  + BloomFilter can be created to enhance performance for queries with precise equal/in conditions. You can find more information about it in BloomFilter datamap [document](https://github.com/apache/carbondata/blob/master/docs/datamap/bloomfilter-datamap-guide.md).
+
+
+## Configuration for Optimizing Data Loading performance for Massive Data
+
+
+  CarbonData supports large data load, in this process sorting data while loading consumes a lot of memory and disk IO and
+  this can result sometimes in "Out Of Memory" exception.
+  If you do not have much memory to use, then you may prefer to slow the speed of data loading instead of data load failure.
+  You can configure CarbonData by tuning following properties in carbon.properties file to get a better performance.
+
+| Parameter | Default Value | Description/Tuning |
+|-----------|-------------|--------|
+|carbon.number.of.cores.while.loading|Default: 2.This value should be >= 2|Specifies the number of cores used for data processing during data loading in CarbonData. |
+|carbon.sort.size|Default: 100000. The value should be >= 100.|Threshold to write local file in sort step when loading data|
+|carbon.sort.file.write.buffer.size|Default:  50000.|DataOutputStream buffer. |
+|carbon.merge.sort.reader.thread|Default: 3 |Specifies the number of cores used for temp file merging during data loading in CarbonData.|
+|carbon.merge.sort.prefetch|Default: true | You may want set this value to false if you have not enough memory|
+
+  For example, if there are 10 million records, and i have only 16 cores, 64GB memory, will be loaded to CarbonData table.
+  Using the default configuration  always fail in sort step. Modify carbon.properties as suggested below:
+
+  ```
+  carbon.merge.sort.reader.thread=1
+  carbon.sort.size=5000
+  carbon.sort.file.write.buffer.size=5000
+  carbon.merge.sort.prefetch=false
+  ```
+
+## Configurations for Optimizing CarbonData Performance
+
+  Recently we did some performance POC on CarbonData for Finance and telecommunication Field. It involved detailed queries and aggregation
+  scenarios. After the completion of POC, some of the configurations impacting the performance have been identified and tabulated below :
+
+| Parameter | Location | Used For  | Description | Tuning |
+|----------------------------------------------|-----------------------------------|---------------------------|----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| carbon.sort.intermediate.files.limit | spark/carbonlib/carbon.properties | Data loading | During the loading of data, local temp is used to sort the data. This number specifies the minimum number of intermediate files after which the  merge sort has to be initiated. | Increasing the parameter to a higher value will improve the load performance. For example, when we increase the value from 20 to 100, it increases the data load performance from 35MB/S to more than 50MB/S. Higher values of this parameter consumes  more memory during the load. |
+| carbon.number.of.cores.while.loading | spark/carbonlib/carbon.properties | Data loading | Specifies the number of cores used for data processing during data loading in CarbonData. | If you have more number of CPUs, then you can increase the number of CPUs, which will increase the performance. For example if we increase the value from 2 to 4 then the CSV reading performance can increase about 1 times |
+| carbon.compaction.level.threshold | spark/carbonlib/carbon.properties | Data loading and Querying | For minor compaction, specifies the number of segments to be merged in stage 1 and number of compacted segments to be merged in stage 2. | Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance. Configuring this parameter will merge the small segment to one big segment which will sort the data and improve the performance. For Example in one telecommunication scenario, the performance improves about 2 times after minor compaction. |
+| spark.sql.shuffle.partitions | spark/conf/spark-defaults.conf | Querying | The number of task started when spark shuffle. | The value can be 1 to 2 times as much as the executor cores. In an aggregation scenario, reducing the number from 200 to 32 reduced the query time from 17 to 9 seconds. |
+| spark.executor.instances/spark.executor.cores/spark.executor.memory | spark/conf/spark-defaults.conf | Querying | The number of executors, CPU cores, and memory used for CarbonData query. | In the bank scenario, we provide the 4 CPUs cores and 15 GB for each executor which can get good performance. This 2 value does not mean more the better. It needs to be configured properly in case of limited resources. For example, In the bank scenario, it has enough CPU 32 cores each node but less memory 64 GB each node. So we cannot give more CPU but less memory. For example, when 4 cores and 12GB for each executor. It sometimes happens GC during the query which impact the query performance very much from the 3 second to more than 15 seconds. In this scenario need to increase the memory or decrease the CPU cores. |
+| carbon.detail.batch.size | spark/carbonlib/carbon.properties | Data loading | The buffer size to store records, returned from the block scan. | In limit scenario this parameter is very important. For example your query limit is 1000. But if we set this value to 3000 that means we get 3000 records from scan but spark will only take 1000 rows. So the 2000 remaining are useless. In one Finance test case after we set it to 100, in the limit 1000 scenario the performance increase about 2 times in comparison to if we set this value to 12000. |
+| carbon.use.local.dir | spark/carbonlib/carbon.properties | Data loading | Whether use YARN local directories for multi-table load disk load balance | If this is set it to true CarbonData will use YARN local directories for multi-table load disk load balance, that will improve the data load performance. |
+| carbon.use.multiple.temp.dir | spark/carbonlib/carbon.properties | Data loading | Whether to use multiple YARN local directories during table data loading for disk load balance | After enabling 'carbon.use.local.dir', if this is set to true, CarbonData will use all YARN local directories during data load for disk load balance, that will improve the data load performance. Please enable this property when you encounter disk hotspot problem during data loading. |
+| carbon.sort.temp.compressor | spark/carbonlib/carbon.properties | Data loading | Specify the name of compressor to compress the intermediate sort temporary files during sort procedure in data loading. | The optional values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files. This parameter will be useful if you encounter disk bottleneck. |
+| carbon.load.skewedDataOptimization.enabled | spark/carbonlib/carbon.properties | Data loading | Whether to enable size based block allocation strategy for data loading. | When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data -- It's useful if the size of your input data files varies widely, say 1MB~1GB. |
+| carbon.load.min.size.enabled | spark/carbonlib/carbon.properties | Data loading | Whether to enable node minumun input data size allocation strategy for data loading.| When loading, carbondata will use node minumun input data size allocation strategy for task distribution. It will make sure the node load the minimum amount of data -- It's useful if the size of your input data files very small, say 1MB~256MB,Avoid generating a large number of small files. |
+
+  Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.
+
+
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__perf').addClass('selected'); });
+</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/preaggregate-datamap-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/preaggregate-datamap-guide.md b/src/site/markdown/preaggregate-datamap-guide.md
index ff4c28e..9c7a5f8 100644
--- a/src/site/markdown/preaggregate-datamap-guide.md
+++ b/src/site/markdown/preaggregate-datamap-guide.md
@@ -126,7 +126,7 @@ kinds of DataMap:
    a. 'path' is used to specify the store location of the datamap.('path'='/location/').
    b. 'partitioning' when set to false enables user to disable partitioning of the datamap.
        Default value is true for this property.
-2. timeseries, for timeseries roll-up table. Please refer to [Timeseries DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/timeseries-datamap-guide.md)
+2. timeseries, for timeseries roll-up table. Please refer to [Timeseries DataMap](./timeseries-datamap-guide.md)
 
 DataMap can be dropped using following DDL
   ```
@@ -271,3 +271,14 @@ release, user can do as following:
 Basically, user can manually trigger the operation by re-building the datamap.
 
 
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/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
index 84f871d..7ac5a3f 100644
--- a/src/site/markdown/quick-start-guide.md
+++ b/src/site/markdown/quick-start-guide.md
@@ -7,7 +7,7 @@
     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.
@@ -16,10 +16,11 @@
 -->
 
 # Quick Start
-This tutorial provides a quick introduction to using CarbonData.
+This tutorial provides a quick introduction to using CarbonData.To follow along with this guide, first download a packaged release of CarbonData from the [CarbonData website](https://dist.apache.org/repos/dist/release/carbondata/).Alternatively it can be created following [Building CarbonData](https://github.com/apache/carbondata/tree/master/build) steps.
 
 ##  Prerequisites
-* [Installation and building CarbonData](https://github.com/apache/carbondata/blob/master/build).
+* Spark 2.2.1 version is installed and running.CarbonData supports Spark versions upto 2.2.1.Please follow steps described in [Spark docs website](https://spark.apache.org/docs/latest) for installing and running Spark.
+
 * Create a sample.csv file using the following commands. The CSV file is required for loading data into CarbonData.
 
   ```
@@ -32,7 +33,30 @@ This tutorial provides a quick introduction to using CarbonData.
   EOF
   ```
 
-## Interactive Analysis with Spark Shell Version 2.1
+## Deployment modes
+
+CarbonData can be integrated with Spark and Presto Execution Engines.The below documentation guides on Installing and Configuring with these execution engines.
+
+### Spark
+
+[Installing and Configuring CarbonData to run locally with Spark Shell](#installing-and-configuring-carbondata-to-run-locally-with-spark-shell)
+
+[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)
+
+
+### Presto
+[Installing and Configuring CarbonData on Presto](#installing-and-configuring-carbondata-on-presto)
+
+
+## Querying Data
+
+[Query Execution using CarbonData Thrift Server](#query-execution-using-carbondata-thrift-server)
+
+## 
+
+## Installing and Configuring CarbonData to run locally with Spark Shell
 
 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.
 
@@ -43,7 +67,7 @@ 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`
+**NOTE**: Path where packaged release of CarbonData was downloaded or 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`.
 
@@ -62,7 +86,7 @@ import org.apache.spark.sql.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)
+**NOTE**: By default metastore location points 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
@@ -86,7 +110,7 @@ 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)
+If you get "tablestatus.lock" issue, please refer to [FAQ](faq.md)
 
 ###### Query Data from a Table
 
@@ -97,3 +121,317 @@ scala>carbon.sql("SELECT city, avg(age), sum(age)
                   FROM test_table
                   GROUP BY city").show()
 ```
+
+
+
+## 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. |
+
+1. 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. If not specified then it takes spark.sql.warehouse.dir path. | hdfs://HOSTNAME:PORT/Opt/CarbonStore | Propose to set HDFS directory |
+
+1. 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.
+
+
+
+## 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/
+```
+
+1. 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` |
+
+1. 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. If not specified then it takes spark.sql.warehouse.dir path. | hdfs://HOSTNAME:PORT/Opt/CarbonStore | Propose to set HDFS directory |
+
+1. 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.
+
+
+
+## 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
+--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. If not specified then it takes spark.sql.warehouse.dir path. | `hdfs://<host_name>:port/user/hive/warehouse/carbon.store` |
+
+**NOTE**: From Spark 1.6, by default the Thrift server runs in multi-session mode. Which means each JDBC/ODBC connection owns a copy of their own SQL configuration and temporary function registry. Cached tables are still shared though. If you prefer to run the Thrift server in single-session mode and share all SQL configuration and temporary function registry, please set option `spark.sql.hive.thriftServer.singleSession` to `true`. You may either add this option to `spark-defaults.conf`, or pass it to `spark-submit.sh` via `--conf`:
+
+```
+./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>
+```
+
+**But** in single-session mode, if one user changes the database from one connection, the database of the other connections will be changed too.
+
+**Examples**
+
+- Start with default memory and executors.
+
+```
+./bin/spark-submit
+--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
+--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
+```
+
+
+
+## Installing and Configuring CarbonData on Presto
+
+
+* ### Installing Presto
+
+ 1. Download the 0.187 version of Presto using:
+    `wget https://repo1.maven.org/maven2/com/facebook/presto/presto-server/0.187/presto-server-0.187.tar.gz`
+
+ 2. Extract Presto tar file: `tar zxvf presto-server-0.187.tar.gz`.
+
+ 3. Download the Presto CLI for the coordinator and name it presto.
+
+  ```
+    wget https://repo1.maven.org/maven2/com/facebook/presto/presto-cli/0.187/presto-cli-0.187-executable.jar
+
+    mv presto-cli-0.187-executable.jar presto
+
+    chmod +x presto
+  ```
+
+### Create Configuration Files
+
+  1. Create `etc` folder in presto-server-0.187 directory.
+  2. Create `config.properties`, `jvm.config`, `log.properties`, and `node.properties` files.
+  3. Install uuid to generate a node.id.
+
+      ```
+      sudo apt-get install uuid
+
+      uuid
+      ```
+
+
+##### Contents of your node.properties file
+
+  ```
+  node.environment=production
+  node.id=<generated uuid>
+  node.data-dir=/home/ubuntu/data
+  ```
+
+##### Contents of your jvm.config file
+
+  ```
+  -server
+  -Xmx16G
+  -XX:+UseG1GC
+  -XX:G1HeapRegionSize=32M
+  -XX:+UseGCOverheadLimit
+  -XX:+ExplicitGCInvokesConcurrent
+  -XX:+HeapDumpOnOutOfMemoryError
+  -XX:OnOutOfMemoryError=kill -9 %p
+  ```
+
+##### Contents of your log.properties file
+  ```
+  com.facebook.presto=INFO
+  ```
+
+ The default minimum level is `INFO`. There are four levels: `DEBUG`, `INFO`, `WARN` and `ERROR`.
+
+### Coordinator Configurations
+
+##### Contents of your config.properties
+  ```
+  coordinator=true
+  node-scheduler.include-coordinator=false
+  http-server.http.port=8086
+  query.max-memory=50GB
+  query.max-memory-per-node=2GB
+  discovery-server.enabled=true
+  discovery.uri=<coordinator_ip>:8086
+  ```
+The options `node-scheduler.include-coordinator=false` and `coordinator=true` indicate that the node is the coordinator and tells the coordinator not to do any of the computation work itself and to use the workers.
+
+**Note**: It is recommended to set `query.max-memory-per-node` to half of the JVM config max memory, though the workload is highly concurrent, lower value for `query.max-memory-per-node` is to be used.
+
+Also relation between below two configuration-properties should be like:
+If, `query.max-memory-per-node=30GB`
+Then, `query.max-memory=<30GB * number of nodes>`.
+
+### Worker Configurations
+
+##### Contents of your config.properties
+
+  ```
+  coordinator=false
+  http-server.http.port=8086
+  query.max-memory=50GB
+  query.max-memory-per-node=2GB
+  discovery.uri=<coordinator_ip>:8086
+  ```
+
+**Note**: `jvm.config` and `node.properties` files are same for all the nodes (worker + coordinator). All the nodes should have different `node.id`.(generated by uuid command).
+
+### Catalog Configurations
+
+1. Create a folder named `catalog` in etc directory of presto on all the nodes of the cluster including the coordinator.
+
+##### Configuring Carbondata in Presto
+1. Create a file named `carbondata.properties` in the `catalog` folder and set the required properties on all the nodes.
+
+### Add Plugins
+
+1. Create a directory named `carbondata` in plugin directory of presto.
+2. Copy `carbondata` jars to `plugin/carbondata` directory on all nodes.
+
+### Start Presto Server on all nodes
+
+```
+./presto-server-0.187/bin/launcher start
+```
+To run it as a background process.
+
+```
+./presto-server-0.187/bin/launcher run
+```
+To run it in foreground.
+
+### Start Presto CLI
+```
+./presto
+```
+To connect to carbondata catalog use the following command:
+
+```
+./presto --server <coordinator_ip>:8086 --catalog carbondata --schema <schema_name>
+```
+Execute the following command to ensure the workers are connected.
+
+```
+select * from system.runtime.nodes;
+```
+Now you can use the Presto CLI on the coordinator to query data sources in the catalog using the Presto workers.
+
+**Note :** Create Tables and data loads should be done before executing queries as we can not create carbon table from this interface.
+
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__quickstart').addClass('selected'); });
+</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/release-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/release-guide.md b/src/site/markdown/release-guide.md
new file mode 100644
index 0000000..40a9058
--- /dev/null
+++ b/src/site/markdown/release-guide.md
@@ -0,0 +1,428 @@
+<!--
+    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.
+-->
+
+# Apache CarbonData Release Guide
+
+Apache CarbonData periodically declares and publishes releases.
+
+Each release is executed by a _Release Manager_, who is selected among the CarbonData committers.
+ This document describes the process that the Release Manager follows to perform a release. Any 
+ changes to this process should be discussed and adopted on the 
+ [dev@ mailing list](mailto:dev@carbondata.apache.org).
+ 
+Please remember that publishing software has legal consequences. This guide complements the 
+foundation-wide [Product Release Policy](http://www.apache.org/dev/release.html) and [Release 
+Distribution Policy](http://www.apache.org/dev/release-distribution).
+
+## Decide to release
+
+Deciding to release and selecting a Release Manager is the first step of the release process. 
+This is a consensus-based decision of the entire community.
+
+Anybody can propose a release on the dev@ mailing list, giving a solid argument and nominating a 
+committer as the Release Manager (including themselves). There's no formal process, no vote 
+requirements, and no timing requirements. Any objections should be resolved by consensus before 
+starting the release.
+
+_Checklist to proceed to next step:_
+
+1. Community agrees to release
+2. Community selects a Release Manager
+
+## Prepare for the release
+
+Before your first release, you should perform one-time configuration steps. This will set up your
+ security keys for signing the artifacts and access release repository.
+ 
+To prepare for each release, you should audit the project status in the Jira, and do necessary 
+bookkeeping. Finally, you should tag a release.
+
+### One-time setup instructions
+
+#### GPG Key
+
+You need to have a GPG key to sign the release artifacts. Please be aware of the ASF-wide 
+[release signing guidelines](https://www.apache.org/dev/release-signing.html). If you don't have 
+a GPG key associated with your Apache account, please create one according to the guidelines.
+
+Determine your Apache GPG key and key ID, as follows:
+
+```
+gpg --list-keys
+```
+
+This will list your GPG keys. One of these should reflect your Apache account, for example:
+
+```
+pub   2048R/845E6689 2016-02-23
+uid                  Nomen Nescio <an...@apache.org>
+sub   2048R/BA4D50BE 2016-02-23
+```
+
+Here, the key ID is the 8-digit hex string in the `pub` line: `845E6689`.
+
+Now, add your Apache GPG key to the CarbonData's `KEYS` file in `dev` and `release` repositories 
+at `dist.apache.org`. Follow the instructions listed at the top of these files.
+ 
+Configure `git` to use this key when signing code by giving it your key ID, as follows:
+
+```
+git config --global user.signingkey 845E6689
+```
+
+You may drop the `--global` option if you'd prefer to use this key for the current repository only.
+
+You may wish to start `gpg-agent` to unlock your GPG key only once using your passphrase. 
+Otherwise, you may need to enter this passphrase several times. The setup of `gpg-agent` varies 
+based on operating system, but may be something like this:
+
+```
+eval $(gpg-agent --daemon --no-grab --write-env-file $HOME/.gpg-agent-info)
+export GPG_TTY=$(tty)
+export GPG_AGENT_INFO
+```
+
+#### Access to Apache Nexus
+
+Configure access to the [Apache Nexus repository](https://repository.apache.org), used for 
+staging repository and promote the artifacts to Maven Central.
+
+1. You log in with your Apache account.
+2. Confirm you have appropriate access by finding `org.apache.carbondata` under `Staging Profiles`.
+3. Navigate to your `Profile` (top right dropdown menu of the page).
+4. Choose `User Token` from the dropdown, then click `Access User Token`. Copy a snippet of the 
+Maven XML configuration block.
+5. Insert this snippet twice into your global Maven `settings.xml` file, typically `${HOME]/
+.m2/settings.xml`. The end result should look like this, where `TOKEN_NAME` and `TOKEN_PASSWORD` 
+are your secret tokens:
+
+```
+ <settings>
+   <servers>
+     <server>
+       <id>apache.releases.https</id>
+       <username>TOKEN_NAME</username>
+       <password>TOKEN_PASSWORD</password>
+     </server>
+     <server>
+       <id>apache.snapshots.https</id>
+       <username>TOKEN_NAME</username>
+       <password>TOKEN_PASSWORD</password>
+     </server>
+   </servers>
+ </settings>
+```
+
+#### Create a new version in Jira
+
+When contributors resolve an issue in Jira, they are tagging it with a release that will contain 
+their changes. With the release currently underway, new issues should be resolved against a 
+subsequent future release. Therefore, you should create a release item for this subsequent 
+release, as follows:
+
+1. In Jira, navigate to `CarbonData > Administration > Versions`.
+2. Add a new release: choose the next minor version number compared to the one currently 
+underway, select today's date as the `Start Date`, and choose `Add`. 
+
+#### Triage release-blocking issues in Jira
+
+There could be outstanding release-blocking issues, which should be triaged before proceeding to 
+build the release. We track them by assigning a specific `Fix Version` field even before the 
+issue is resolved.
+
+The list of release-blocking issues is available at the [version status page](https://issues.apache.org/jira/browse/CARBONDATA/?selectedTab=com.atlassian.jira.jira-projects-plugin:versions-panel). 
+Triage each unresolved issue with one of the following resolutions:
+
+* If the issue has been resolved and Jira was not updated, resolve it accordingly.
+* If the issue has not been resolved and it is acceptable to defer until the next release, update
+ the `Fix Version` field to the new version you just created. Please consider discussing this 
+ with stakeholders and the dev@ mailing list, as appropriate.
+* If the issue has not been resolved and it is not acceptable to release until it is fixed, the 
+ release cannot proceed. Instead, work with the CarbonData community to resolve the issue.
+ 
+#### Review Release Notes in Jira
+
+Jira automatically generates Release Notes based on the `Fix Version` applied to the issues. 
+Release Notes are intended for CarbonData users (not CarbonData committers/contributors). You 
+should ensure that Release Notes are informative and useful.
+
+Open the release notes from the [version status page](https://issues.apache.org/jira/browse/CARBONDATA/?selectedTab=com.atlassian.jira.jira-projects-plugin:versions-panel)
+by choosing the release underway and clicking Release Notes.
+
+You should verify that the issues listed automatically by Jira are appropriate to appear in the 
+Release Notes. Specifically, issues should:
+
+* Be appropriate classified as `Bug`, `New Feature`, `Improvement`, etc.
+* Represent noteworthy user-facing changes, such as new functionality, backward-incompatible 
+changes, or performance improvements.
+* Have occurred since the previous release; an issue that was introduced and fixed between 
+releases should not appear in the Release Notes.
+* Have an issue title that makes sense when read on its own.
+
+Adjust any of the above properties to the improve clarity and presentation of the Release Notes.
+
+#### Verify that a Release Build works
+
+Run `mvn clean install -Prelease` to ensure that the build processes that are specific to that 
+profile are in good shape.
+
+_Checklist to proceed to the next step:_
+
+1. Release Manager's GPG key is published to `dist.apache.org`.
+2. Release Manager's GPG key is configured in `git` configuration.
+3. Release Manager has `org.apache.carbondata` listed under `Staging Profiles` in Nexus.
+4. Release Manager's Nexus User Token is configured in `settings.xml`.
+5. Jira release item for the subsequent release has been created.
+6. There are no release blocking Jira issues.
+7. Release Notes in Jira have been audited and adjusted.
+
+### Build a release
+
+Use Maven release plugin to tag and build release artifacts, as follows:
+
+```
+mvn release:prepare
+```
+
+Use Maven release plugin to stage these artifacts on the Apache Nexus repository, as follows:
+
+```
+mvn release:perform
+```
+
+Review all staged artifacts. They should contain all relevant parts for each module, including 
+`pom.xml`, jar, test jar, source, etc. Artifact names should follow 
+[the existing format](https://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.carbondata%22)
+in which artifact name mirrors directory structure. Carefully review any new artifacts.
+
+Close the staging repository on Nexus. When prompted for a description, enter "Apache CarbonData 
+x.x.x release".
+
+### Stage source release on dist.apache.org
+
+Copy the source release to dev repository on `dist.apache.org`.
+
+1. If you have not already, check out the section of the `dev` repository on `dist.apache.org` via Subversion. In a fresh directory:
+
+```
+svn co https://dist.apache.org/repos/dist/dev/carbondata
+```
+
+2. Make a directory for the new release:
+
+```
+mkdir x.x.x
+```
+
+3. Copy the CarbonData source distribution, hash, and GPG signature:
+
+```
+cp apache-carbondata-x.x.x-source-release.zip x.x.x
+```
+
+4. Add and commit the files:
+
+```
+svn add x.x.x
+svn commit
+```
+
+5. Verify the files are [present](https://dist.apache.org/repos/dist/dev/carbondata).
+
+### Propose a pull request for website updates
+
+The final step of building a release candidate is to propose a website pull request.
+
+This pull request should update the following page with the new release:
+
+* `src/main/webapp/index.html`
+* `src/main/webapp/docs/latest/mainpage.html`
+
+_Checklist to proceed to the next step:_
+
+1. Maven artifacts deployed to the staging repository of 
+[repository.apache.org](https://repository.apache.org)
+2. Source distribution deployed to the dev repository of
+[dist.apache.org](https://dist.apache.org/repos/dist/dev/carbondata/)
+3. Website pull request to list the release.
+
+## Vote on the release candidate
+
+Once you have built and individually reviewed the release candidate, please share it for the 
+community-wide review. Please review foundation-wide [voting guidelines](http://www.apache.org/foundation/voting.html)
+for more information.
+
+Start the review-and-vote thread on the dev@ mailing list. Here's an email template; please 
+adjust as you see fit:
+
+```
+From: Release Manager
+To: dev@carbondata.apache.org
+Subject: [VOTE] Apache CarbonData Release x.x.x
+
+Hi everyone,
+Please review and vote on the release candidate for the version x.x.x, as follows:
+
+[ ] +1, Approve the release
+[ ] -1, Do not approve the release (please provide specific comments)
+
+The complete staging area is available for your review, which includes:
+* JIRA release notes [1],
+* the official Apache source release to be deployed to dist.apache.org [2], which is signed with the key with fingerprint FFFFFFFF [3],
+* all artifacts to be deployed to the Maven Central Repository [4],
+* source code tag "x.x.x" [5],
+* website pull request listing the release [6].
+
+The vote will be open for at least 72 hours. It is adopted by majority approval, with at least 3 PMC affirmative votes.
+
+Thanks,
+Release Manager
+
+[1] link
+[2] link
+[3] https://dist.apache.org/repos/dist/dist/carbondata/KEYS
+[4] link
+[5] link
+[6] link
+```
+
+If there are any issues found in the release candidate, reply on the vote thread to cancel the vote.
+There’s no need to wait 72 hours. Proceed to the `Cancel a Release (Fix Issues)` step below and 
+address the problem.
+However, some issues don’t require cancellation.
+For example, if an issue is found in the website pull request, just correct it on the spot and the
+vote can continue as-is.
+
+If there are no issues, reply on the vote thread to close the voting. Then, tally the votes in a
+separate email. Here’s an email template; please adjust as you see fit.
+
+```
+From: Release Manager
+To: dev@carbondata.apache.org
+Subject: [RESULT][VOTE] Apache CarbonData Release x.x.x
+
+I'm happy to announce that we have unanimously approved this release.
+
+There are XXX approving votes, XXX of which are binding:
+* approver 1
+* approver 2
+* approver 3
+* approver 4
+
+There are no disapproving votes.
+
+Thanks everyone!
+```
+
+While in incubation, the Apache Incubator PMC must also vote on each release, using the same 
+process as above. Start the review and vote thread on the `general@incubator.apache.org` list.
+
+
+_Checklist to proceed to the final step:_
+
+1. Community votes to release the proposed release
+2. While in incubation, Apache Incubator PMC votes to release the proposed release
+
+## Cancel a Release (Fix Issues)
+
+Any issue identified during the community review and vote should be fixed in this step.
+
+To fully cancel a vote:
+
+* Cancel the current release and verify the version is back to the correct SNAPSHOT:
+
+```
+mvn release:cancel
+```
+
+* Drop the release tag:
+
+```
+git tag -d x.x.x
+git push --delete apache x.x.x
+```
+
+* Drop the staging repository on Nexus ([repository.apache.org](https://repository.apache.org))
+
+
+Verify the version is back to the correct SNAPSHOT.
+
+Code changes should be proposed as standard pull requests and merged.
+
+Once all issues have been resolved, you should go back and build a new release candidate with 
+these changes.
+
+## Finalize the release
+
+Once the release candidate has been reviewed and approved by the community, the release should be
+ finalized. This involves the final deployment of the release to the release repositories, 
+ merging the website changes, and announce the release.
+ 
+### Deploy artifacts to Maven Central repository
+
+On Nexus, release the staged artifacts to Maven Central repository. In the `Staging Repositories`
+ section, find the relevant release candidate `orgapachecarbondata-XXX` entry and click `Release`.
+
+### Deploy source release to dist.apache.org
+
+Copy the source release from the `dev` repository to `release` repository at `dist.apache.org` 
+using Subversion.
+
+### Merge website pull request
+
+Merge the website pull request to list the release created earlier.
+
+### Mark the version as released in Jira
+
+In Jira, inside [version management](https://issues.apache.org/jira/plugins/servlet/project-config/CARBONDATA/versions)
+, hover over the current release and a settings menu will appear. Click `Release`, and select 
+today's state.
+
+_Checklist to proceed to the next step:_
+
+1. Maven artifacts released and indexed in the
+ [Maven Central repository](https://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.carbondata%22)
+2. Source distribution available in the release repository of
+ [dist.apache.org](https://dist.apache.org/repos/dist/release/carbondata/)
+3. Website pull request to list the release merged
+4. Release version finalized in Jira
+
+## Promote the release
+
+Once the release has been finalized, the last step of the process is to promote the release 
+within the project and beyond.
+
+### Apache mailing lists
+
+Announce on the dev@ mailing list that the release has been finished.
+ 
+Announce on the user@ mailing list that the release is available, listing major improvements and 
+contributions.
+
+While in incubation, announce the release on the Incubator's general@ mailing list.
+
+_Checklist to declare the process completed:_
+
+1. Release announced on the user@ mailing list.
+2. Release announced on the Incubator's general@ mailing list.
+3. Completion declared on the dev@ mailing list.
+
+
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__release').addClass('selected'); });
+</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/s3-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/s3-guide.md b/src/site/markdown/s3-guide.md
index 2f4dfa9..37f157c 100644
--- a/src/site/markdown/s3-guide.md
+++ b/src/site/markdown/s3-guide.md
@@ -15,7 +15,7 @@
     limitations under the License.
 -->
 
-#S3 Guide (Alpha Feature 1.4.1)
+# S3 Guide (Alpha Feature 1.4.1)
 
 Object storage is the recommended storage format in cloud as it can support storing large data 
 files. S3 APIs are widely used for accessing object stores. This can be 
@@ -26,7 +26,7 @@ data and the data can be accessed from anywhere at any time.
 Carbondata can support any Object Storage that conforms to Amazon S3 API.
 Carbondata relies on Hadoop provided S3 filesystem APIs to access Object stores.
 
-#Writing to Object Storage
+# Writing to Object Storage
 
 To store carbondata files onto Object Store, `carbon.storelocation` property will have 
 to be configured with Object Store path in CarbonProperties file. 
@@ -46,9 +46,9 @@ For example:
 CREATE TABLE IF NOT EXISTS db1.table1(col1 string, col2 int) STORED AS carbondata LOCATION 's3a://mybucket/carbonstore'
 ``` 
 
-For more details on create table, Refer [data-management-on-carbondata](./data-management-on-carbondata.md#create-table)
+For more details on create table, Refer [DDL of CarbonData](ddl-of-carbondata.md#create-table)
 
-#Authentication
+# Authentication
 
 Authentication properties will have to be configured to store the carbondata files on to S3 location. 
 
@@ -80,12 +80,15 @@ sparkSession.sparkContext.hadoopConfiguration.set("fs.s3a.secret.key", "123")
 sparkSession.sparkContext.hadoopConfiguration.set("fs.s3a.access.key","456")
 ```
 
-#Recommendations
+# Recommendations
 
 1. Object Storage like S3 does not support file leasing mechanism(supported by HDFS) that is 
 required to take locks which ensure consistency between concurrent operations therefore, it is 
-recommended to set the configurable lock path property([carbon.lock.path](https://github.com/apache/carbondata/blob/master/docs/configuration-parameters.md#miscellaneous-configuration))
+recommended to set the configurable lock path property([carbon.lock.path](./configuration-parameters.md#system-configuration))
  to a HDFS directory.
-2. Concurrent data manipulation operations are not supported. Object stores follow eventual 
-consistency semantics, i.e., any put request might take some time to reflect when trying to list
-.This behaviour causes not to ensure the data read is always consistent or latest.
+2. Concurrent data manipulation operations are not supported. Object stores follow eventual consistency semantics, i.e., any put request might take some time to reflect when trying to list. This behaviour causes the data read is always not consistent or not the latest.
+
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__s3').addClass('selected'); });
+</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/sdk-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/sdk-guide.md b/src/site/markdown/sdk-guide.md
index e592aa5..66f3d61 100644
--- a/src/site/markdown/sdk-guide.md
+++ b/src/site/markdown/sdk-guide.md
@@ -351,6 +351,25 @@ public CarbonWriterBuilder withLoadOptions(Map<String, String> options);
 
 ```
 /**
+ * To support the table properties for sdk writer
+ *
+ * @param options key,value pair of create table properties.
+ * supported keys values are
+ * a. blocksize -- [1-2048] values in MB. Default value is 1024
+ * b. blockletsize -- values in MB. Default value is 64 MB
+ * c. localDictionaryThreshold -- positive value, default is 10000
+ * d. enableLocalDictionary -- true / false. Default is false
+ * e. sortcolumns -- comma separated column. "c1,c2". Default all dimensions are sorted.
+ *
+ * @return updated CarbonWriterBuilder
+ */
+public CarbonWriterBuilder withTableProperties(Map<String, String> options);
+```
+
+
+```
+/**
+* this writer is not thread safe, use buildThreadSafeWriterForCSVInput in multi thread environment
 * Build a {@link CarbonWriter}, which accepts row in CSV format object
 * @param schema carbon Schema object {org.apache.carbondata.sdk.file.Schema}
 * @return CSVCarbonWriter
@@ -360,8 +379,24 @@ public CarbonWriterBuilder withLoadOptions(Map<String, String> options);
 public CarbonWriter buildWriterForCSVInput(org.apache.carbondata.sdk.file.Schema schema) throws IOException, InvalidLoadOptionException;
 ```
 
+```
+/**
+* Can use this writer in multi-thread instance.
+* Build a {@link CarbonWriter}, which accepts row in CSV format
+* @param schema carbon Schema object {org.apache.carbondata.sdk.file.Schema}
+* @param numOfThreads number of threads() in which .write will be called.              
+* @return CSVCarbonWriter
+* @throws IOException
+* @throws InvalidLoadOptionException
+*/
+public CarbonWriter buildThreadSafeWriterForCSVInput(Schema schema, short numOfThreads)
+  throws IOException, InvalidLoadOptionException;
+```
+
+
 ```  
 /**
+* this writer is not thread safe, use buildThreadSafeWriterForAvroInput in multi thread environment
 * Build a {@link CarbonWriter}, which accepts Avro format object
 * @param avroSchema avro Schema object {org.apache.avro.Schema}
 * @return AvroCarbonWriter 
@@ -373,6 +408,22 @@ public CarbonWriter buildWriterForAvroInput(org.apache.avro.Schema schema) throw
 
 ```
 /**
+* Can use this writer in multi-thread instance.
+* Build a {@link CarbonWriter}, which accepts Avro object
+* @param avroSchema avro Schema object {org.apache.avro.Schema}
+* @param numOfThreads number of threads() in which .write will be called.
+* @return AvroCarbonWriter
+* @throws IOException
+* @throws InvalidLoadOptionException
+*/
+public CarbonWriter buildThreadSafeWriterForAvroInput(org.apache.avro.Schema avroSchema, short numOfThreads)
+  throws IOException, InvalidLoadOptionException
+```
+
+
+```
+/**
+* this writer is not thread safe, use buildThreadSafeWriterForJsonInput in multi thread environment
 * Build a {@link CarbonWriter}, which accepts Json object
 * @param carbonSchema carbon Schema object
 * @return JsonCarbonWriter
@@ -382,6 +433,19 @@ public CarbonWriter buildWriterForAvroInput(org.apache.avro.Schema schema) throw
 public JsonCarbonWriter buildWriterForJsonInput(Schema carbonSchema);
 ```
 
+```
+/**
+* Can use this writer in multi-thread instance.
+* Build a {@link CarbonWriter}, which accepts Json object
+* @param carbonSchema carbon Schema object
+* @param numOfThreads number of threads() in which .write will be called.
+* @return JsonCarbonWriter
+* @throws IOException
+* @throws InvalidLoadOptionException
+*/
+public JsonCarbonWriter buildThreadSafeWriterForJsonInput(Schema carbonSchema, short numOfThreads)
+```
+
 ### Class org.apache.carbondata.sdk.file.CarbonWriter
 ```
 /**
@@ -390,7 +454,7 @@ public JsonCarbonWriter buildWriterForJsonInput(Schema carbonSchema);
 *                      which is one row of data.
 * If CSVCarbonWriter, object is of type String[], which is one row of data
 * If JsonCarbonWriter, object is of type String, which is one row of json
-* Note: This API is not thread safe
+* Note: This API is not thread safe if writer is not built with number of threads argument.
 * @param object
 * @throws IOException
 */
@@ -678,7 +742,6 @@ Find example code at [CarbonReaderExample](https://github.com/apache/carbondata/
    *
    * @param dataFilePath complete path including carbondata file name
    * @return Schema object
-   * @throws IOException
    */
   public static Schema readSchemaInDataFile(String dataFilePath);
 ```
@@ -802,4 +865,10 @@ public String getProperty(String key);
 */
 public String getProperty(String key, String defaultValue);
 ```
-Reference : [list of carbon properties](http://carbondata.apache.org/configuration-parameters.html)
+Reference : [list of carbon properties](./configuration-parameters.md)
+
+
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__api').addClass('selected'); });
+</script>


[39/39] carbondata-site git commit: refactor document

Posted by ch...@apache.org.
refactor document


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

Branch: refs/heads/asf-site
Commit: bee563340a6b03838789e8802f76a3f38ab2cf61
Parents: ca81b94
Author: chenliang613 <ch...@huawei.com>
Authored: Sat Sep 8 00:53:30 2018 +0800
Committer: chenliang613 <ch...@huawei.com>
Committed: Sat Sep 8 00:53:30 2018 +0800

----------------------------------------------------------------------
 content/data-management-on-carbondata.html | 1321 +++++++++++++++++++++++
 content/data-management.html               |  413 +++++++
 content/ddl-operation-on-carbondata.html   |  748 +++++++++++++
 content/dml-operation-on-carbondata.html   |  716 ++++++++++++
 content/installation-guide.html            |  455 ++++++++
 content/mainpage.html                      |  214 ++++
 content/sdk-writer-guide.html              |  549 ++++++++++
 content/troubleshooting.html               |  366 +++++++
 content/useful-tips-on-carbondata.html     |  480 ++++++++
 9 files changed, 5262 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/bee56334/content/data-management-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/data-management-on-carbondata.html b/content/data-management-on-carbondata.html
new file mode 100644
index 0000000..566bb8e
--- /dev/null
+++ b/content/data-management-on-carbondata.html
@@ -0,0 +1,1321 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="row">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="data-management-on-carbondata" class="anchor" href="#data-management-on-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Management on CarbonData</h1>
+<p>This tutorial is going to introduce all commands and data operations on CarbonData.</p>
+<ul>
+<li><a href="#create-table">CREATE TABLE</a></li>
+<li><a href="#create-database">CREATE DATABASE</a></li>
+<li><a href="#table-management">TABLE MANAGEMENT</a></li>
+<li><a href="#load-data">LOAD DATA</a></li>
+<li><a href="#update-and-delete">UPDATE AND DELETE</a></li>
+<li><a href="#compaction">COMPACTION</a></li>
+<li><a href="#partition">PARTITION</a></li>
+<li><a href="#bucketing">BUCKETING</a></li>
+<li><a href="#segment-management">SEGMENT MANAGEMENT</a></li>
+</ul>
+<h2>
+<a id="create-table" class="anchor" href="#create-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE</h2>
+<p>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.</p>
+<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type , ...)]
+STORED AS carbondata
+[TBLPROPERTIES (property_name=property_value, ...)]
+[LOCATION 'path']
+</code></pre>
+<p><strong>NOTE:</strong> CarbonData also supports "STORED AS carbondata" and "USING carbondata". Find example code at <a href="https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala" target=_blank>CarbonSessionExample</a> in the CarbonData repo.</p>
+<h3>
+<a id="usage-guidelines" class="anchor" href="#usage-guidelines" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
+<p>Following are the guidelines for TBLPROPERTIES, CarbonData's additional table options can be set via carbon.properties.</p>
+<ul>
+<li>
+<p><strong>Dictionary Encoding Configuration</strong></p>
+<p>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.</p>
+<pre><code>TBLPROPERTIES ('DICTIONARY_INCLUDE'='column1, column2')
+</code></pre>
+<p>NOTE: Dictionary Include/Exclude for complex child columns is not supported.</p>
+</li>
+<li>
+<p><strong>Inverted Index Configuration</strong></p>
+<p>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.</p>
+<pre><code>TBLPROPERTIES ('NO_INVERTED_INDEX'='column1, column3')
+</code></pre>
+</li>
+<li>
+<p><strong>Sort Columns Configuration</strong></p>
+<p>This property is for users to specify which columns belong to the MDK(Multi-Dimensions-Key) index.</p>
+<ul>
+<li>If users don't specify "SORT_COLUMN" property, by default MDK index be built by using all dimension columns except complex data type column.</li>
+<li>If this property is specified but with empty argument, then the table will be loaded without sort.</li>
+<li>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.</li>
+</ul>
+<pre><code>TBLPROPERTIES ('SORT_COLUMNS'='column1, column3')
+OR
+TBLPROPERTIES ('SORT_COLUMNS'='')
+</code></pre>
+<p>NOTE: Sort_Columns for Complex datatype columns is not supported.</p>
+</li>
+<li>
+<p><strong>Sort Scope Configuration</strong></p>
+<p>This property is for users to specify the scope of the sort during data load, following are the types of sort scope.</p>
+<ul>
+<li>LOCAL_SORT: It is the default sort scope.</li>
+<li>NO_SORT: It will load the data in unsorted manner, it will significantly increase load performance.</li>
+<li>BATCH_SORT: It increases the load performance but decreases the query performance if identified blocks &gt; parallelism.</li>
+<li>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.</li>
+</ul>
+</li>
+</ul>
+<pre><code>### Example:
+</code></pre>
+<pre><code> 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 ('SORT_COLUMNS'='productName,storeCity',
+                'SORT_SCOPE'='NO_SORT')
+</code></pre>
+<p><strong>NOTE:</strong> CarbonData also supports "using carbondata". Find example code at <a href="https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala" target=_blank>SparkSessionExample</a> in the CarbonData repo.</p>
+<ul>
+<li>
+<p><strong>Table Block Size Configuration</strong></p>
+<p>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.</p>
+<pre><code>TBLPROPERTIES ('TABLE_BLOCKSIZE'='512')
+</code></pre>
+<p><strong>NOTE:</strong> 512 or 512M both are accepted.</p>
+</li>
+<li>
+<p><strong>Table Compaction Configuration</strong></p>
+<p>These properties are table level compaction configurations, if not specified, system level configurations in carbon.properties will be used.
+Following are 5 configurations:</p>
+<ul>
+<li>MAJOR_COMPACTION_SIZE: same meaning as carbon.major.compaction.size, size in MB.</li>
+<li>AUTO_LOAD_MERGE: same meaning as carbon.enable.auto.load.merge.</li>
+<li>COMPACTION_LEVEL_THRESHOLD: same meaning as carbon.compaction.level.threshold.</li>
+<li>COMPACTION_PRESERVE_SEGMENTS: same meaning as carbon.numberof.preserve.segments.</li>
+<li>ALLOWED_COMPACTION_DAYS: same meaning as carbon.allowed.compaction.days.</li>
+</ul>
+<pre><code>TBLPROPERTIES ('MAJOR_COMPACTION_SIZE'='2048',
+               'AUTO_LOAD_MERGE'='true',
+               'COMPACTION_LEVEL_THRESHOLD'='5,6',
+               'COMPACTION_PRESERVE_SEGMENTS'='10',
+               'ALLOWED_COMPACTION_DAYS'='5')
+</code></pre>
+</li>
+<li>
+<p><strong>Streaming</strong></p>
+<p>CarbonData supports streaming ingestion for real-time data. You can create the ?streaming? table using the following table properties.</p>
+<pre><code>TBLPROPERTIES ('streaming'='true')
+</code></pre>
+</li>
+<li>
+<p><strong>Local Dictionary Configuration</strong></p>
+</li>
+</ul>
+<p>Columns for which dictionary is not generated needs more storage space and in turn more IO. Also since more data will have to be read during query, query performance also would suffer.Generating dictionary per blocklet for such columns would help in saving storage space and assist in improving query performance as carbondata is optimized for handling dictionary encoded columns more effectively.Generating dictionary internally per blocklet is termed as local dictionary. Please refer to <a href="../file-structure-of-carbondata.html">File structure of Carbondata</a> for understanding about the file structure of carbondata and meaning of terms like blocklet.</p>
+<p>Local Dictionary helps in:</p>
+<ol>
+<li>Getting more compression.</li>
+<li>Filter queries and full scan queries will be faster as filter will be done on encoded data.</li>
+<li>Reducing the store size and memory footprint as only unique values will be stored as part of local dictionary and corresponding data will be stored as encoded data.</li>
+<li>Getting higher IO throughput.</li>
+</ol>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>
+<p>Following Data Types are Supported for Local Dictionary:</p>
+<ul>
+<li>STRING</li>
+<li>VARCHAR</li>
+<li>CHAR</li>
+</ul>
+</li>
+<li>
+<p>Following Data Types are not Supported for Local Dictionary:</p>
+<ul>
+<li>SMALLINT</li>
+<li>INTEGER</li>
+<li>BIGINT</li>
+<li>DOUBLE</li>
+<li>DECIMAL</li>
+<li>TIMESTAMP</li>
+<li>DATE</li>
+<li>BOOLEAN</li>
+</ul>
+</li>
+<li>
+<p>In case of multi-level complex dataType columns, primitive string/varchar/char columns are considered for local dictionary generation.</p>
+</li>
+</ul>
+<p>Local dictionary will have to be enabled explicitly during create table or by enabling the system property 'carbon.local.dictionary.enable'. By default, Local Dictionary will be disabled for the carbondata table.</p>
+<p>Local Dictionary can be configured using the following properties during create table command:</p>
+<table>
+<thead>
+<tr>
+<th>Properties</th>
+<th>Default value</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>LOCAL_DICTIONARY_ENABLE</td>
+<td>false</td>
+<td>Whether to enable local dictionary generation. <strong>NOTE:</strong> If this property is defined, it will override the value configured at system level by 'carbon.local.dictionary.enable'</td>
+</tr>
+<tr>
+<td>LOCAL_DICTIONARY_THRESHOLD</td>
+<td>10000</td>
+<td>The maximum cardinality of a column upto which carbondata can try to generate local dictionary (maximum - 100000)</td>
+</tr>
+<tr>
+<td>LOCAL_DICTIONARY_INCLUDE</td>
+<td>string/varchar/char columns</td>
+<td>Columns for which Local Dictionary has to be generated.<strong>NOTE:</strong> Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.</td>
+</tr>
+<tr>
+<td>LOCAL_DICTIONARY_EXCLUDE</td>
+<td>none</td>
+<td>Columns for which Local Dictionary need not be generated.</td>
+</tr>
+</tbody>
+</table>
+<p><strong>Fallback behavior:</strong></p>
+<ul>
+<li>When the cardinality of a column exceeds the threshold, it triggers a fallback and the generated dictionary will be reverted and data loading will be continued without dictionary encoding.</li>
+</ul>
+<p><strong>NOTE:</strong> When fallback is triggered, the data loading performance will decrease as encoded data will be discarded and the actual data is written to the temporary sort files.</p>
+<p><strong>Points to be noted:</strong></p>
+<ol>
+<li>
+<p>Reduce Block size:</p>
+<p>Number of Blocks generated is less in case of Local Dictionary as compression ratio is high. This may reduce the number of tasks launched during query, resulting in degradation of query performance if the pruned blocks are less compared to the number of parallel tasks which can be run. So it is recommended to configure smaller block size which in turn generates more number of blocks.</p>
+</li>
+<li>
+<p>All the page-level data for a blocklet needs to be maintained in memory until all the pages encoded for local dictionary is processed in order to handle fallback. Hence the memory required for local dictionary based table is more and this memory increase is proportional to number of columns.</p>
+</li>
+</ol>
+<h3>
+<a id="example" class="anchor" href="#example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
+<pre><code>CREATE TABLE carbontable(
+          
+            column1 string,
+          
+            column2 string,
+          
+            column3 LONG )
+          
+  STORED BY 'carbondata'
+  TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='true','LOCAL_DICTIONARY_THRESHOLD'='1000',
+  'LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>We recommend to use Local Dictionary when cardinality is high but is distributed across multiple loads</li>
+<li>On a large cluster, decoding data can become a bottleneck for global dictionary as there will be many remote reads. In this scenario, it is better to use Local Dictionary.</li>
+<li>When cardinality is less, but loads are repetitive, it is better to use global dictionary as local dictionary generates multiple dictionary files at blocklet level increasing redundancy.</li>
+</ul>
+<ul>
+<li>
+<p><strong>Caching Min/Max Value for Required Columns</strong>
+By default, CarbonData caches min and max values of all the columns in schema.  As the load increases, the memory required to hold the min and max values increases considerably. This feature enables you to configure min and max values only for the required columns, resulting in optimized memory usage.</p>
+<p>Following are the valid values for COLUMN_META_CACHE:</p>
+<ul>
+<li>If you want no column min/max values to be cached in the driver.</li>
+</ul>
+<pre><code>COLUMN_META_CACHE=??
+</code></pre>
+<ul>
+<li>If you want only col1 min/max values to be cached in the driver.</li>
+</ul>
+<pre><code>COLUMN_META_CACHE=?col1?
+</code></pre>
+<ul>
+<li>If you want min/max values to be cached in driver for all the specified columns.</li>
+</ul>
+<pre><code>COLUMN_META_CACHE=?col1,col2,col3,??
+</code></pre>
+<p>Columns to be cached can be specified either while creating table or after creation of the table.
+During create table operation; specify the columns to be cached in table properties.</p>
+<p>Syntax:</p>
+<pre><code>CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,?) STORED BY ?carbondata? TBLPROPERTIES (?COLUMN_META_CACHE?=?col1,col2,??)
+</code></pre>
+<p>Example:</p>
+<pre><code>CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES (?COLUMN_META_CACHE?=?name?)
+</code></pre>
+<p>After creation of table or on already created tables use the alter table command to configure the columns to be cached.</p>
+<p>Syntax:</p>
+<pre><code>ALTER TABLE [dbName].tableName SET TBLPROPERTIES (?COLUMN_META_CACHE?=?col1,col2,??)
+</code></pre>
+<p>Example:</p>
+<pre><code>ALTER TABLE employee SET TBLPROPERTIES (?COLUMN_META_CACHE?=?city?)
+</code></pre>
+</li>
+<li>
+<p><strong>Caching at Block or Blocklet Level</strong></p>
+<p>This feature allows you to maintain the cache at Block level, resulting in optimized usage of the memory. The memory consumption is high if the Blocklet level caching is maintained as a Block can have multiple Blocklet.</p>
+<p>Following are the valid values for CACHE_LEVEL:</p>
+<p><em>Configuration for caching in driver at Block level (default value).</em></p>
+<pre><code>CACHE_LEVEL= ?BLOCK?
+</code></pre>
+<p><em>Configuration for caching in driver at Blocklet level.</em></p>
+<pre><code>CACHE_LEVEL= ?BLOCKLET?
+</code></pre>
+<p>Cache level can be specified either while creating table or after creation of the table.
+During create table operation specify the cache level in table properties.</p>
+<p>Syntax:</p>
+<pre><code>CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,?) STORED BY ?carbondata? TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
+</code></pre>
+<p>Example:</p>
+<pre><code>CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
+</code></pre>
+<p>After creation of table or on already created tables use the alter table command to configure the cache level.</p>
+<p>Syntax:</p>
+<pre><code>ALTER TABLE [dbName].tableName SET TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
+</code></pre>
+<p>Example:</p>
+<pre><code>ALTER TABLE employee SET TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
+</code></pre>
+</li>
+</ul>
+<pre><code>- **Support Flat folder same as Hive/Parquet**
+
+  This feature allows all carbondata and index files to keep directy under tablepath. Currently all carbondata/carbonindex files written under tablepath/Fact/Part0/Segment_NUM folder and it is not same as hive/parquet folder structure. This feature makes all files written will be directly under tablepath, it does not maintain any segment folder structure.This is useful for interoperability between the execution engines and plugin with other execution engines like hive or presto becomes easier.
+
+  Following table property enables this feature and default value is false.
+  ```
+   'flat_folder'='true'
+  ```
+  Example:
+  ```
+  CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES ('flat_folder'='true')
+  ```
+
+- **String longer than 32000 characters**
+
+ In common scenarios, the length of string is less than 32000,
+ so carbondata stores the length of content using Short to reduce memory and space consumption.
+ To support string longer than 32000 characters, carbondata introduces a table property called `LONG_STRING_COLUMNS`.
+ For these columns, carbondata internally stores the length of content using Integer.
+
+ You can specify the columns as 'long string column' using below tblProperties:
+
+ ```
+ // specify col1, col2 as long string columns
+ TBLPROPERTIES ('LONG_STRING_COLUMNS'='col1,col2')
+ ```
+
+ Besides, you can also use this property through DataFrame by
+ ```
+ df.format("carbondata")
+   .option("tableName", "carbonTable")
+   .option("long_string_columns", "col1, col2")
+   .save()
+ ```
+
+ If you are using Carbon-SDK, you can specify the datatype of long string column as `varchar`.
+ You can refer to SDKwriterTestCase for example.
+
+ **NOTE:** The LONG_STRING_COLUMNS can only be string/char/varchar columns and cannot be dictionary_include/sort_columns/complex columns.
+</code></pre>
+<h2>
+<a id="create-table-as-select" class="anchor" href="#create-table-as-select" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE AS SELECT</h2>
+<p>This function allows user 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.</p>
+<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name 
+STORED BY 'carbondata' 
+[TBLPROPERTIES (key1=val1, key2=val2, ...)] 
+AS select_statement;
+</code></pre>
+<h3>
+<a id="examples" class="anchor" href="#examples" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples</h3>
+<pre><code>carbon.sql("CREATE TABLE source_table(
+                           id INT,
+                           name STRING,
+                           city STRING,
+                           age INT)
+            STORED AS parquet")
+carbon.sql("INSERT INTO source_table SELECT 1,'bob','shenzhen',27")
+carbon.sql("INSERT INTO source_table SELECT 2,'david','shenzhen',31")
+
+carbon.sql("CREATE TABLE target_table
+            STORED BY 'carbondata'
+            AS SELECT city,avg(age) FROM source_table GROUP BY city")
+            
+carbon.sql("SELECT * FROM target_table").show
+  // results:
+  //    +--------+--------+
+  //    |    city|avg(age)|
+  //    +--------+--------+
+  //    |shenzhen|    29.0|
+  //    +--------+--------+
+
+</code></pre>
+<h2>
+<a id="create-external-table" class="anchor" href="#create-external-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE EXTERNAL TABLE</h2>
+<p>This function allows user to create external table by specifying location.</p>
+<pre><code>CREATE EXTERNAL TABLE [IF NOT EXISTS] [db_name.]table_name 
+STORED BY 'carbondata' LOCATION ?$FilesPath?
+</code></pre>
+<h3>
+<a id="create-external-table-on-managed-table-data-location" class="anchor" href="#create-external-table-on-managed-table-data-location" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create external table on managed table data location.</h3>
+<p>Managed table data location provided will have both FACT and Metadata folder.
+This data can be generated by creating a normal carbon table and use this path as $FilesPath in the above syntax.</p>
+<p><strong>Example:</strong></p>
+<pre><code>sql("CREATE TABLE origin(key INT, value STRING) STORED BY 'carbondata'")
+sql("INSERT INTO origin select 100,'spark'")
+sql("INSERT INTO origin select 200,'hive'")
+// creates a table in $storeLocation/origin
+
+sql(s"""
+|CREATE EXTERNAL TABLE source
+|STORED BY 'carbondata'
+|LOCATION '$storeLocation/origin'
+""".stripMargin)
+checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin"))
+</code></pre>
+<h3>
+<a id="create-external-table-on-non-transactional-table-data-location" class="anchor" href="#create-external-table-on-non-transactional-table-data-location" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create external table on Non-Transactional table data location.</h3>
+<p>Non-Transactional table data location will have only carbondata and carbonindex files, there will not be a metadata folder (table status and schema).
+Our SDK module currently support writing data in this format.</p>
+<p><strong>Example:</strong></p>
+<pre><code>sql(
+s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbondata' LOCATION
+|'$writerPath' """.stripMargin)
+</code></pre>
+<p>Here writer path will have carbondata and index files.
+This can be SDK output. Refer <a href="https://github.com/apache/carbondata/blob/master/docs/sdk-writer-guide.html" target=_blank>SDK Writer Guide</a>.</p>
+<p><strong>Note:</strong></p>
+<ol>
+<li>Dropping of the external table should not delete the files present in the location.</li>
+<li>When external table is created on non-transactional table data,
+external table will be registered with the schema of carbondata files.
+If multiple files with different schema is present, exception will be thrown.
+So, If table registered with one schema and files are of different schema,
+suggest to drop the external table and create again to register table with new schema.</li>
+</ol>
+<h2>
+<a id="create-database" class="anchor" href="#create-database" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE DATABASE</h2>
+<p>This function creates a new database. By default the database is created in Carbon store location, but you can also specify custom location.</p>
+<pre><code>CREATE DATABASE [IF NOT EXISTS] database_name [LOCATION path];
+</code></pre>
+<h3>
+<a id="example-1" class="anchor" href="#example-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example</h3>
+<pre><code>CREATE DATABASE carbon LOCATION ?hdfs://name_cluster/dir1/carbonstore?;
+</code></pre>
+<h2>
+<a id="table-management" class="anchor" href="#table-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>TABLE MANAGEMENT</h2>
+<h3>
+<a id="show-table" class="anchor" href="#show-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW TABLE</h3>
+<p>This command can be used to list all the tables in current database or all the tables of a specific database.</p>
+<pre><code>SHOW TABLES [IN db_Name]
+</code></pre>
+<p>Example:</p>
+<pre><code>SHOW TABLES
+OR
+SHOW TABLES IN defaultdb
+</code></pre>
+<h3>
+<a id="alter-table" class="anchor" href="#alter-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>ALTER TABLE</h3>
+<p>The following section introduce the commands to modify the physical or logical state of the existing table(s).</p>
+<ul>
+<li>
+<p><strong>RENAME TABLE</strong></p>
+<p>This command is used to rename the existing table.</p>
+<pre><code>ALTER TABLE [db_name.]table_name RENAME TO new_table_name
+</code></pre>
+<p>Examples:</p>
+<pre><code>ALTER TABLE carbon RENAME TO carbonTable
+OR
+ALTER TABLE test_db.carbon RENAME TO test_db.carbonTable
+</code></pre>
+</li>
+<li>
+<p><strong>ADD COLUMNS</strong></p>
+<p>This command is used to add a new column to the existing table.</p>
+<pre><code>ALTER TABLE [db_name.]table_name ADD COLUMNS (col_name data_type,...)
+TBLPROPERTIES('DICTIONARY_INCLUDE'='col_name,...',
+'DEFAULT.VALUE.COLUMN_NAME'='default_value')
+</code></pre>
+<p>Examples:</p>
+<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING)
+</code></pre>
+<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DICTIONARY_INCLUDE'='a1')
+</code></pre>
+<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DEFAULT.VALUE.a1'='10')
+</code></pre>
+<p>NOTE: Add Complex datatype columns is not supported.</p>
+</li>
+</ul>
+<p>Users can specify which columns to include and exclude for local dictionary generation after adding new columns. These will be appended with the already existing local dictionary include and exclude columns of main table respectively.</p>
+<pre><code>   ALTER TABLE carbon ADD COLUMNS (a1 STRING, b1 STRING) TBLPROPERTIES('LOCAL_DICTIONARY_INCLUDE'='a1','LOCAL_DICTIONARY_EXCLUDE'='b1')
+</code></pre>
+<ul>
+<li>
+<p><strong>DROP COLUMNS</strong></p>
+<p>This command is used to delete the existing column(s) in a table.</p>
+<pre><code>ALTER TABLE [db_name.]table_name DROP COLUMNS (col_name, ...)
+</code></pre>
+<p>Examples:</p>
+<pre><code>ALTER TABLE carbon DROP COLUMNS (b1)
+OR
+ALTER TABLE test_db.carbon DROP COLUMNS (b1)
+
+ALTER TABLE carbon DROP COLUMNS (c1,d1)
+</code></pre>
+<p>NOTE: Drop Complex child column is not supported.</p>
+</li>
+<li>
+<p><strong>CHANGE DATA TYPE</strong></p>
+<p>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.</p>
+<pre><code>ALTER TABLE [db_name.]table_name CHANGE col_name col_name changed_column_type
+</code></pre>
+<p>Valid Scenarios</p>
+<ul>
+<li>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.</li>
+<li>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.</li>
+<li>
+<strong>NOTE:</strong> The allowed range is 38,38 (precision, scale) and is a valid upper case scenario which is not resulting in data loss.</li>
+</ul>
+<p>Example1:Changing data type of column a1 from INT to BIGINT.</p>
+<pre><code>ALTER TABLE test_db.carbon CHANGE a1 a1 BIGINT
+</code></pre>
+<p>Example2:Changing decimal precision of column a1 from 10 to 18.</p>
+<pre><code>ALTER TABLE test_db.carbon CHANGE a1 a1 DECIMAL(18,2)
+</code></pre>
+</li>
+<li>
+<p><strong>MERGE INDEX</strong></p>
+<p>This command is used to merge all the CarbonData index files (.carbonindex) inside a segment to a single CarbonData index merge file (.carbonindexmerge). This enhances the first query performance.</p>
+<pre><code> ALTER TABLE [db_name.]table_name COMPACT 'SEGMENT_INDEX'
+ ```
+ 
+ Examples:
+ ```
+ ALTER TABLE test_db.carbon COMPACT 'SEGMENT_INDEX'
+ ```
+ **NOTE:**
+ * Merge index is not supported on streaming table.
+ 
+</code></pre>
+</li>
+<li>
+<p><strong>SET and UNSET for Local Dictionary Properties</strong></p>
+<p>When set command is used, all the newly set properties will override the corresponding old properties if exists.</p>
+<p>Example to SET Local Dictionary Properties:</p>
+<pre><code>ALTER TABLE tablename SET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='false','LOCAL_DICTIONARY_THRESHOLD'='1000','LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
+</code></pre>
+<p>When Local Dictionary properties are unset, corresponding default values will be used for these properties.</p>
+<p>Example to UNSET Local Dictionary Properties:</p>
+<pre><code>ALTER TABLE tablename UNSET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE','LOCAL_DICTIONARY_THRESHOLD','LOCAL_DICTIONARY_INCLUDE','LOCAL_DICTIONARY_EXCLUDE')
+</code></pre>
+<p><strong>NOTE:</strong> For old tables, by default, local dictionary is disabled. If user wants local dictionary for these tables, user can enable/disable local dictionary for new data at their discretion.
+This can be achieved by using the alter table set command.</p>
+</li>
+</ul>
+<h3>
+<a id="drop-table" class="anchor" href="#drop-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DROP TABLE</h3>
+<p>This command is used to delete an existing table.</p>
+<pre><code>DROP TABLE [IF EXISTS] [db_name.]table_name
+</code></pre>
+<p>Example:</p>
+<pre><code>DROP TABLE IF EXISTS productSchema.productSalesTable
+</code></pre>
+<h3>
+<a id="refresh-table" class="anchor" href="#refresh-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>REFRESH TABLE</h3>
+<p>This command is used to register Carbon table to HIVE meta store catalogue from existing Carbon table data.</p>
+<pre><code>REFRESH TABLE $db_NAME.$table_NAME
+</code></pre>
+<p>Example:</p>
+<pre><code>REFRESH TABLE dbcarbon.productSalesTable
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>The new database name and the old database name should be same.</li>
+<li>Before executing this command the old table schema and data should be copied into the new database location.</li>
+<li>If the table is aggregate table, then all the aggregate tables should be copied to the new database location.</li>
+<li>For old store, the time zone of the source and destination cluster should be same.</li>
+<li>If old cluster used HIVE meta store to store schema, refresh will not work as schema file does not exist in file system.</li>
+</ul>
+<h3>
+<a id="table-and-column-comment" class="anchor" href="#table-and-column-comment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Table and Column Comment</h3>
+<p>You can provide more information on table by using table comment. Similarly you can provide more information about a particular column using column comment.
+You can see the column comment of an existing table using describe formatted command.</p>
+<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type [COMMENT col_comment], ...)]
+  [COMMENT table_comment]
+STORED BY 'carbondata'
+[TBLPROPERTIES (property_name=property_value, ...)]
+</code></pre>
+<p>Example:</p>
+<pre><code>CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
+                              productNumber Int COMMENT 'unique serial number for product')
+COMMENT ?This is table comment?
+ STORED BY 'carbondata'
+ TBLPROPERTIES ('DICTIONARY_INCLUDE'='productNumber')
+</code></pre>
+<p>You can also SET and UNSET table comment using ALTER command.</p>
+<p>Example to SET table comment:</p>
+<pre><code>ALTER TABLE carbon SET TBLPROPERTIES ('comment'='this table comment is modified');
+</code></pre>
+<p>Example to UNSET table comment:</p>
+<pre><code>ALTER TABLE carbon UNSET TBLPROPERTIES ('comment');
+</code></pre>
+<h2>
+<a id="load-data" class="anchor" href="#load-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LOAD DATA</h2>
+<h3>
+<a id="load-files-to-carbondata-table" class="anchor" href="#load-files-to-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LOAD FILES TO CARBONDATA TABLE</h3>
+<p>This command is used to load csv files to carbondata, OPTIONS are not mandatory for data loading process.
+Inside OPTIONS user can provide any options like DELIMITER, QUOTECHAR, FILEHEADER, ESCAPECHAR, MULTILINE as per requirement.</p>
+<pre><code>LOAD DATA [LOCAL] INPATH 'folder_path' 
+INTO TABLE [db_name.]table_name 
+OPTIONS(property_name=property_value, ...)
+</code></pre>
+<p>You can use the following options to load data:</p>
+<ul>
+<li>
+<p><strong>DELIMITER:</strong> Delimiters can be provided in the load command.</p>
+<pre><code>OPTIONS('DELIMITER'=',')
+</code></pre>
+</li>
+<li>
+<p><strong>QUOTECHAR:</strong> Quote Characters can be provided in the load command.</p>
+<pre><code>OPTIONS('QUOTECHAR'='"')
+</code></pre>
+</li>
+<li>
+<p><strong>COMMENTCHAR:</strong> Comment Characters can be provided in the load command if user want to comment lines.</p>
+<pre><code>OPTIONS('COMMENTCHAR'='#')
+</code></pre>
+</li>
+<li>
+<p><strong>HEADER:</strong> When you load the CSV file without the file header and the file header is the same with the table schema, then add 'HEADER'='false' to load data SQL as user need not provide the file header. By default the value is 'true'.
+false: CSV file is without file header.
+true: CSV file is with file header.</p>
+<pre><code>OPTIONS('HEADER'='false') 
+</code></pre>
+<p><strong>NOTE:</strong> If the HEADER option exist and is set to 'true', then the FILEHEADER option is not required.</p>
+</li>
+<li>
+<p><strong>FILEHEADER:</strong> Headers can be provided in the LOAD DATA command if headers are missing in the source files.</p>
+<pre><code>OPTIONS('FILEHEADER'='column1,column2') 
+</code></pre>
+</li>
+<li>
+<p><strong>MULTILINE:</strong> CSV with new line character in quotes.</p>
+<pre><code>OPTIONS('MULTILINE'='true') 
+</code></pre>
+</li>
+<li>
+<p><strong>ESCAPECHAR:</strong> Escape char can be provided if user want strict validation of escape character in CSV files.</p>
+<pre><code>OPTIONS('ESCAPECHAR'='\') 
+</code></pre>
+</li>
+<li>
+<p><strong>SKIP_EMPTY_LINE:</strong> This option will ignore the empty line in the CSV file during the data load.</p>
+<pre><code>OPTIONS('SKIP_EMPTY_LINE'='TRUE/FALSE') 
+</code></pre>
+</li>
+<li>
+<p><strong>COMPLEX_DELIMITER_LEVEL_1:</strong> Split the complex type data column in a row (eg., a$b$c --&gt; Array = {a,b,c}).</p>
+<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_1'='$') 
+</code></pre>
+</li>
+<li>
+<p><strong>COMPLEX_DELIMITER_LEVEL_2:</strong> Split the complex type nested data column in a row. Applies level_1 delimiter &amp; applies level_2 based on complex data type (eg., a:b$c:d --&gt; Array&gt; = {{a,b},{c,d}}).</p>
+<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_2'=':')
+</code></pre>
+</li>
+<li>
+<p><strong>ALL_DICTIONARY_PATH:</strong> All dictionary files path.</p>
+<pre><code>OPTIONS('ALL_DICTIONARY_PATH'='/opt/alldictionary/data.dictionary')
+</code></pre>
+</li>
+<li>
+<p><strong>COLUMNDICT:</strong> Dictionary file path for specified column.</p>
+<pre><code>OPTIONS('COLUMNDICT'='column1:dictionaryFilePath1,column2:dictionaryFilePath2')
+</code></pre>
+<p><strong>NOTE:</strong> ALL_DICTIONARY_PATH and COLUMNDICT can't be used together.</p>
+</li>
+<li>
+<p><strong>DATEFORMAT/TIMESTAMPFORMAT:</strong> Date and Timestamp format for specified column.</p>
+<pre><code>OPTIONS('DATEFORMAT' = 'yyyy-MM-dd','TIMESTAMPFORMAT'='yyyy-MM-dd HH:mm:ss')
+</code></pre>
+<p><strong>NOTE:</strong> Date formats are specified by date pattern strings. The date pattern letters in CarbonData are same as in JAVA. Refer to <a href="http://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html" target=_blank rel="nofollow">SimpleDateFormat</a>.</p>
+</li>
+<li>
+<p><strong>SORT COLUMN BOUNDS:</strong> Range bounds for sort columns.</p>
+<p>Suppose the table is created with 'SORT_COLUMNS'='name,id' and the range for name is aaa<del>zzz, the value range for id is 0</del>1000. Then during data loading, we can specify the following option to enhance data loading performance.</p>
+<pre><code>OPTIONS('SORT_COLUMN_BOUNDS'='f,250;l,500;r,750')
+</code></pre>
+<p>Each bound is separated by ';' and each field value in bound is separated by ','. In the example above, we provide 3 bounds to distribute records to 4 partitions. The values 'f','l','r' can evenly distribute the records. Inside carbondata, for a record we compare the value of sort columns with that of the bounds and decide which partition the record will be forwarded to.</p>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>SORT_COLUMN_BOUNDS will be used only when the SORT_SCOPE is 'local_sort'.</li>
+<li>Carbondata will use these bounds as ranges to process data concurrently during the final sort percedure. The records will be sorted and written out inside each partition. Since the partition is sorted, all records will be sorted.</li>
+<li>Since the actual order and literal order of the dictionary column are not necessarily the same, we do not recommend you to use this feature if the first sort column is 'dictionary_include'.</li>
+<li>The option works better if your CPU usage during loading is low. If your system is already CPU tense, better not to use this option. Besides, it depends on the user to specify the bounds. If user does not know the exactly bounds to make the data distributed evenly among the bounds, loading performance will still be better than before or at least the same as before.</li>
+<li>Users can find more information about this option in the description of PR1953.</li>
+</ul>
+</li>
+<li>
+<p><strong>SINGLE_PASS:</strong> 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.</p>
+</li>
+</ul>
+<p>This option specifies whether to use single pass for loading data or not. By default this option is set to FALSE.</p>
+<pre><code> OPTIONS('SINGLE_PASS'='TRUE')
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>If this option is set to TRUE then data loading will take less time.</li>
+<li>If this option is set to some invalid value other than TRUE or FALSE then it uses the default value.</li>
+</ul>
+<p>Example:</p>
+<pre><code>LOAD DATA local inpath '/opt/rawdata/data.csv' INTO table carbontable
+options('DELIMITER'=',', 'QUOTECHAR'='"','COMMENTCHAR'='#',
+'HEADER'='false',
+'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')
+</code></pre>
+<ul>
+<li>
+<p><strong>BAD RECORDS HANDLING:</strong> Methods of handling bad records are as follows:</p>
+<ul>
+<li>Load all of the data before dealing with the errors.</li>
+<li>Clean or delete bad records before loading data or stop the loading when bad records are found.</li>
+</ul>
+<pre><code>OPTIONS('BAD_RECORDS_LOGGER_ENABLE'='true', 'BAD_RECORD_PATH'='hdfs://hacluster/tmp/carbon', 'BAD_RECORDS_ACTION'='REDIRECT', 'IS_EMPTY_DATA_BAD_RECORD'='false')
+</code></pre>
+</li>
+</ul>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>BAD_RECORDS_ACTION property can have four type of actions for bad records FORCE, REDIRECT, IGNORE and FAIL.</li>
+<li>FAIL option is its Default value. If the FAIL option is used, then data loading fails if any bad records are found.</li>
+<li>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.</li>
+<li>If the FORCE option is used, then it auto-converts the data by storing the bad records as NULL before Loading data.</li>
+<li>If the IGNORE option is used, then bad records are neither loaded nor written to the separate CSV file.</li>
+<li>In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails.</li>
+<li>The default maximum number of characters per column is 32000. If there are more than 32000 characters in a column, please refer to <em>String longer than 32000 characters</em> section.</li>
+<li>Since Bad Records Path can be specified in create, load and carbon properties.
+Therefore, value specified in load will have the highest priority, and value specified in carbon properties will have the least priority.</li>
+</ul>
+<p><strong>Bad Records Path:</strong></p>
+<p>This property is used to specify the location where bad records would be written.</p>
+<pre><code>TBLPROPERTIES('BAD_RECORDS_PATH'='/opt/badrecords'')
+</code></pre>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<ul>
+<li>
+<strong>GLOBAL_SORT_PARTITIONS:</strong> If the SORT_SCOPE is defined as GLOBAL_SORT, then user can specify the number of partitions to use while shuffling data for sort using GLOBAL_SORT_PARTITIONS. If it is not configured, or configured less than 1, then it uses the number of map task as reduce task. It is recommended that each reduce task deal with 512MB-1GB data.</li>
+</ul>
+<pre><code>OPTIONS('GLOBAL_SORT_PARTITIONS'='2')
+</code></pre>
+<p>NOTE:</p>
+<ul>
+<li>GLOBAL_SORT_PARTITIONS should be Integer type, the range is [1,Integer.MaxValue].</li>
+<li>It is only used when the SORT_SCOPE is GLOBAL_SORT.</li>
+</ul>
+<h3>
+<a id="insert-data-into-carbondata-table" class="anchor" href="#insert-data-into-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>INSERT DATA INTO CARBONDATA TABLE</h3>
+<p>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.</p>
+<pre><code>INSERT INTO TABLE &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p>You can also omit the <code>table</code> keyword and write your query as:</p>
+<pre><code>INSERT INTO &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p>Overwrite insert data:</p>
+<pre><code>INSERT OVERWRITE TABLE &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>The source table and the CarbonData table must have the same table schema.</li>
+<li>The data type of source and destination table columns should be same</li>
+<li>INSERT INTO command does not support partial success if bad records are found, it will fail.</li>
+<li>Data cannot be loaded or updated in source table while insert from source table to target table is in progress.</li>
+</ul>
+<p>Examples</p>
+<pre><code>INSERT INTO table1 SELECT item1, sum(item2 + 1000) as result FROM table2 group by item1
+</code></pre>
+<pre><code>INSERT INTO table1 SELECT item1, item2, item3 FROM table2 where item2='xyz'
+</code></pre>
+<pre><code>INSERT OVERWRITE TABLE table1 SELECT * FROM TABLE2
+</code></pre>
+<h2>
+<a id="update-and-delete" class="anchor" href="#update-and-delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>UPDATE AND DELETE</h2>
+<h3>
+<a id="update" class="anchor" href="#update" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>UPDATE</h3>
+<p>This command will allow to update the CarbonData table based on the column expression and optional filter conditions.</p>
+<pre><code>UPDATE &lt;table_name&gt; 
+SET (column_name1, column_name2, ... column_name n) = (column1_expression , column2_expression, ... column n_expression )
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p>alternatively the following command can also be used for updating the CarbonData Table :</p>
+<pre><code>UPDATE &lt;table_name&gt;
+SET (column_name1, column_name2) =(select sourceColumn1, sourceColumn2 from sourceTable [ WHERE { &lt;filter_condition&gt; } ] )
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p><strong>NOTE:</strong> The update command fails if multiple input rows in source table are matched with single row in destination table.</p>
+<p>Examples:</p>
+<pre><code>UPDATE t3 SET (t3_salary) = (t3_salary + 9) WHERE t3_name = 'aaa1'
+</code></pre>
+<pre><code>UPDATE t3 SET (t3_date, t3_country) = ('2017-11-18', 'india') WHERE t3_salary &lt; 15003
+</code></pre>
+<pre><code>UPDATE t3 SET (t3_country, t3_name) = (SELECT t5_country, t5_name FROM t5 WHERE t5_id = 5) WHERE t3_id &lt; 5
+</code></pre>
+<pre><code>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 &lt; 5
+</code></pre>
+<pre><code>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 &gt;6
+</code></pre>
+<p>NOTE: Update Complex datatype columns is not supported.</p>
+<h3>
+<a id="delete" class="anchor" href="#delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE</h3>
+<p>This command allows us to delete records from CarbonData table.</p>
+<pre><code>DELETE FROM table_name [WHERE expression]
+</code></pre>
+<p>Examples:</p>
+<pre><code>DELETE FROM carbontable WHERE column1  = 'china'
+</code></pre>
+<pre><code>DELETE FROM carbontable WHERE column1 IN ('china', 'USA')
+</code></pre>
+<pre><code>DELETE FROM carbontable WHERE column1 IN (SELECT column11 FROM sourceTable2)
+</code></pre>
+<pre><code>DELETE FROM carbontable WHERE column1 IN (SELECT column11 FROM sourceTable2 WHERE column1 = 'USA')
+</code></pre>
+<h2>
+<a id="compaction" class="anchor" href="#compaction" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COMPACTION</h2>
+<p>Compaction improves the query performance significantly.</p>
+<p>There are several types of compaction.</p>
+<pre><code>ALTER TABLE [db_name.]table_name COMPACT 'MINOR/MAJOR/CUSTOM'
+</code></pre>
+<ul>
+<li><strong>Minor Compaction</strong></li>
+</ul>
+<p>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:</p>
+<ul>
+<li>Level 1: Merging of the segments which are not yet compacted.</li>
+<li>Level 2: Merging of the compacted segments again to form a larger segment.</li>
+</ul>
+<pre><code>ALTER TABLE table_name COMPACT 'MINOR'
+</code></pre>
+<ul>
+<li><strong>Major Compaction</strong></li>
+</ul>
+<p>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.
+Configure the property carbon.major.compaction.size with appropriate value in MB.</p>
+<p>This command merges the specified number of segments into one segment:</p>
+<pre><code>ALTER TABLE table_name COMPACT 'MAJOR'
+</code></pre>
+<ul>
+<li><strong>Custom Compaction</strong></li>
+</ul>
+<p>In Custom compaction, user can directly specify segment ids to be merged into one large segment.
+All specified segment ids should exist and be valid, otherwise compaction will fail.
+Custom compaction is usually done during the off-peak time.</p>
+<pre><code>ALTER TABLE table_name COMPACT 'CUSTOM' WHERE SEGMENT.ID IN (2,3,4)
+</code></pre>
+<p>NOTE: Compaction is unsupported for table containing Complex columns.</p>
+<ul>
+<li><strong>CLEAN SEGMENTS AFTER Compaction</strong></li>
+</ul>
+<p>Clean the segments which are compacted:</p>
+<pre><code>CLEAN FILES FOR TABLE carbon_table
+</code></pre>
+<h2>
+<a id="partition" class="anchor" href="#partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>PARTITION</h2>
+<h3>
+<a id="standard-partition" class="anchor" href="#standard-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>STANDARD PARTITION</h3>
+<p>The partition is similar as spark and hive partition, user can use any column to build partition:</p>
+<h4>
+<a id="create-partition-table" class="anchor" href="#create-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Partition Table</h4>
+<p>This command allows you to create table with partition.</p>
+<pre><code>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, ...)]
+</code></pre>
+<p>Example:</p>
+<pre><code> 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'
+</code></pre>
+<p>NOTE: Hive partition is not supported on complex datatype columns.</p>
+<h4>
+<a id="load-data-using-static-partition" class="anchor" href="#load-data-using-static-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Load Data Using Static Partition</h4>
+<p>This command allows you to load data using static partition.</p>
+<pre><code>LOAD DATA [LOCAL] INPATH 'folder_path' 
+INTO TABLE [db_name.]table_name PARTITION (partition_spec) 
+OPTIONS(property_name=property_value, ...)    
+INSERT INTO INTO TABLE [db_name.]table_name PARTITION (partition_spec) &lt;SELECT STATEMENT&gt;
+</code></pre>
+<p>Example:</p>
+<pre><code>LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.csv'
+INTO TABLE locationTable
+PARTITION (country = 'US', state = 'CA')  
+INSERT INTO TABLE locationTable
+PARTITION (country = 'US', state = 'AL')
+SELECT &lt;columns list excluding partition columns&gt; FROM another_user
+</code></pre>
+<h4>
+<a id="load-data-using-dynamic-partition" class="anchor" href="#load-data-using-dynamic-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Load Data Using Dynamic Partition</h4>
+<p>This command allows you to load data using dynamic partition. If partition spec is not specified, then the partition is considered as dynamic.</p>
+<p>Example:</p>
+<pre><code>LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.csv'
+INTO TABLE locationTable          
+INSERT INTO TABLE locationTable
+SELECT &lt;columns list excluding partition columns&gt; FROM another_user
+</code></pre>
+<h4>
+<a id="show-partitions" class="anchor" href="#show-partitions" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Show Partitions</h4>
+<p>This command gets the Hive partition information of the table</p>
+<pre><code>SHOW PARTITIONS [db_name.]table_name
+</code></pre>
+<h4>
+<a id="drop-partition" class="anchor" href="#drop-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Drop Partition</h4>
+<p>This command drops the specified Hive partition only.</p>
+<pre><code>ALTER TABLE table_name DROP [IF EXISTS] PARTITION (part_spec, ...)
+</code></pre>
+<p>Example:</p>
+<pre><code>ALTER TABLE locationTable DROP PARTITION (country = 'US');
+</code></pre>
+<h4>
+<a id="insert-overwrite" class="anchor" href="#insert-overwrite" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Insert OVERWRITE</h4>
+<p>This command allows you to insert or load overwrite on a specific partition.</p>
+<pre><code> INSERT OVERWRITE TABLE table_name
+ PARTITION (column = 'partition_name')
+ select_statement
+</code></pre>
+<p>Example:</p>
+<pre><code>INSERT OVERWRITE TABLE partitioned_user
+PARTITION (country = 'US')
+SELECT * FROM another_user au 
+WHERE au.country = 'US';
+</code></pre>
+<h3>
+<a id="carbondata-partitionhashrangelist----alpha-feature-this-partition-feature-does-not-support-update-and-delete-data" class="anchor" href="#carbondata-partitionhashrangelist----alpha-feature-this-partition-feature-does-not-support-update-and-delete-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CARBONDATA PARTITION(HASH,RANGE,LIST) -- Alpha feature, this partition feature does not support update and delete data.</h3>
+<p>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.</p>
+<h3>
+<a id="create-hash-partition-table" class="anchor" href="#create-hash-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Hash Partition Table</h3>
+<p>This command allows us to create hash partition.</p>
+<pre><code>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' ...)]
+</code></pre>
+<p><strong>NOTE:</strong> N is the number of hash partitions</p>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<h3>
+<a id="create-range-partition-table" class="anchor" href="#create-range-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Range Partition Table</h3>
+<p>This command allows us to create range partition.</p>
+<pre><code>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, ...')]
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>The 'RANGE_INFO' must be defined in ascending order in the table properties.</li>
+<li>The default format for partition column of Date/Timestamp type is yyyy-MM-dd. Alternate formats for Date/Timestamp could be defined in CarbonProperties.</li>
+</ul>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<h3>
+<a id="create-list-partition-table" class="anchor" href="#create-list-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create List Partition Table</h3>
+<p>This command allows us to create list partition.</p>
+<pre><code>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, ...')]
+</code></pre>
+<p><strong>NOTE:</strong> List partition supports list info in one level group.</p>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<h3>
+<a id="show-partitions-1" class="anchor" href="#show-partitions-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Show Partitions</h3>
+<p>The following command is executed to get the partition information of the table</p>
+<pre><code>SHOW PARTITIONS [db_name.]table_name
+</code></pre>
+<h3>
+<a id="add-a-new-partition" class="anchor" href="#add-a-new-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Add a new partition</h3>
+<pre><code>ALTER TABLE [db_name].table_name ADD PARTITION('new_partition')
+</code></pre>
+<h3>
+<a id="split-a-partition" class="anchor" href="#split-a-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Split a partition</h3>
+<pre><code>ALTER TABLE [db_name].table_name SPLIT PARTITION(partition_id) INTO('new_partition1', 'new_partition2'...)
+</code></pre>
+<h3>
+<a id="drop-a-partition" class="anchor" href="#drop-a-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Drop a partition</h3>
+<p>Only drop partition definition, but keep data</p>
+<pre><code>  ALTER TABLE [db_name].table_name DROP PARTITION(partition_id)
+</code></pre>
+<p>Drop both partition definition and data</p>
+<pre><code>ALTER TABLE [db_name].table_name DROP PARTITION(partition_id) WITH DATA
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>Hash partition table is not supported for ADD, SPLIT and DROP commands.</li>
+<li>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.</li>
+</ul>
+<pre><code>SegmentDir/0_batchno0-0-1502703086921.carbonindex
+          ^
+SegmentDir/part-0-0_batchno0-0-1502703086921.carbondata
+                   ^
+</code></pre>
+<p>Here are some useful tips to improve query performance of carbonData partition table:</p>
+<ul>
+<li>The partitioned column can be excluded from SORT_COLUMNS, this will let other columns to do the efficient sorting.</li>
+<li>When writing SQL on a partition table, try to use filters on the partition column.</li>
+</ul>
+<h2>
+<a id="bucketing" class="anchor" href="#bucketing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>BUCKETING</h2>
+<p>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.</p>
+<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
+                  [(col_name data_type, ...)]
+STORED BY 'carbondata'
+TBLPROPERTIES('BUCKETNUMBER'='noOfBuckets',
+'BUCKETCOLUMNS'='columnname')
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>Bucketing cannot be performed for columns of Complex Data Types.</li>
+<li>Columns in the BUCKETCOLUMN parameter must be dimensions. The BUCKETCOLUMN parameter cannot be a measure or a combination of measures and dimensions.</li>
+</ul>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<h2>
+<a id="segment-management" class="anchor" href="#segment-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SEGMENT MANAGEMENT</h2>
+<h3>
+<a id="show-segment" class="anchor" href="#show-segment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW SEGMENT</h3>
+<p>This command is used to list the segments of CarbonData table.</p>
+<pre><code>SHOW [HISTORY] SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
+</code></pre>
+<p>Example:
+Show visible segments</p>
+<pre><code>SHOW SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4
+</code></pre>
+<p>Show all segments, include invisible segments</p>
+<pre><code>SHOW HISTORY SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4
+</code></pre>
+<h3>
+<a id="delete-segment-by-id" class="anchor" href="#delete-segment-by-id" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY ID</h3>
+<p>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.</p>
+<p>The following command will get the segmentID.</p>
+<pre><code>SHOW SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
+</code></pre>
+<p>After you retrieve the segment ID of the segment that you want to delete, execute the following command to delete the selected segment.</p>
+<pre><code>DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.ID IN (segment_id1, segments_id2, ...)
+</code></pre>
+<p>Example:</p>
+<pre><code>DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0)
+DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0,5,8)
+</code></pre>
+<h3>
+<a id="delete-segment-by-date" class="anchor" href="#delete-segment-by-date" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY DATE</h3>
+<p>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.</p>
+<pre><code>DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.STARTTIME BEFORE DATE_VALUE
+</code></pre>
+<p>Example:</p>
+<pre><code>DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.STARTTIME BEFORE '2017-06-01 12:05:06' 
+</code></pre>
+<h3>
+<a id="query-data-with-specified-segments" class="anchor" href="#query-data-with-specified-segments" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>QUERY DATA WITH SPECIFIED SEGMENTS</h3>
+<p>This command is used to read data from specified segments during CarbonScan.</p>
+<p>Get the Segment ID:</p>
+<pre><code>SHOW SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
+</code></pre>
+<p>Set the segment IDs for table</p>
+<pre><code>SET carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt; = &lt;list of segment IDs&gt;
+</code></pre>
+<p><strong>NOTE:</strong>
+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.</p>
+<p>If user wants to query with segments reading in multi threading mode, then CarbonSession. threadSet can be used instead of SET query.</p>
+<pre><code>CarbonSession.threadSet ("carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt;","&lt;list of segment IDs&gt;");
+</code></pre>
+<p>Reset the segment IDs</p>
+<pre><code>SET carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt; = *;
+</code></pre>
+<p>If user wants to query with segments reading in multi threading mode, then CarbonSession. threadSet can be used instead of SET query.</p>
+<pre><code>CarbonSession.threadSet ("carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt;","*");
+</code></pre>
+<p><strong>Examples:</strong></p>
+<ul>
+<li>Example to show the list of segment IDs,segment status, and other required details and then specify the list of segments to be read.</li>
+</ul>
+<pre><code>SHOW SEGMENTS FOR carbontable1;
+
+SET carbon.input.segments.db.carbontable1 = 1,3,9;
+</code></pre>
+<ul>
+<li>Example to query with segments reading in multi threading mode:</li>
+</ul>
+<pre><code>CarbonSession.threadSet ("carbon.input.segments.db.carbontable_Multi_Thread","1,3");
+</code></pre>
+<ul>
+<li>Example for threadset in multithread environment (following shows how it is used in Scala code):</li>
+</ul>
+<pre><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();
+   }
+ }
+</code></pre>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file


[15/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/how-to-contribute-to-apache-carbondata.html
----------------------------------------------------------------------
diff --git a/content/how-to-contribute-to-apache-carbondata.html b/content/how-to-contribute-to-apache-carbondata.html
new file mode 100644
index 0000000..946013f
--- /dev/null
+++ b/content/how-to-contribute-to-apache-carbondata.html
@@ -0,0 +1,371 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="how-to-contribute-to-apache-carbondata" class="anchor" href="#how-to-contribute-to-apache-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>How to contribute to Apache CarbonData</h1>
+<p>The Apache CarbonData community welcomes all kinds of contributions from anyone with a passion for
+faster data format! Apache CarbonData is a new file format for faster interactive query using
+advanced columnar storage, index, compression and encoding techniques to improve computing
+efficiency,in turn it will help speedup queries an order of magnitude faster over PetaBytes of data.</p>
+<p>We use a review-then-commit workflow in CarbonData for all contributions.</p>
+<ul>
+<li>Engage -&gt; Design -&gt; Code -&gt; Review -&gt; Commit</li>
+</ul>
+<h2>
+<a id="engage" class="anchor" href="#engage" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Engage</h2>
+<h3>
+<a id="mailing-lists" class="anchor" href="#mailing-lists" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Mailing list(s)</h3>
+<p>We discuss design and implementation issues on <a href="mailto:dev@carbondata.apache.org">dev@carbondata.apache.org</a> Join by
+emailing <a href="mailto:dev-subscribe@carbondata.apache.org">dev-subscribe@carbondata.apache.org</a></p>
+<h3>
+<a id="apache-jira" class="anchor" href="#apache-jira" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Apache JIRA</h3>
+<p>We use <a href="https://issues.apache.org/jira/browse/CARBONDATA" target=_blank rel="nofollow">Apache JIRA</a> as an issue tracking and
+project management tool, as well as a way to communicate among a very diverse and distributed set
+of contributors. To be able to gather feedback, avoid frustration, and avoid duplicated efforts all
+CarbonData-related work should be tracked there.</p>
+<p>If you do not already have an Apache JIRA account, sign up <a href="https://issues.apache.org/jira/" target=_blank rel="nofollow">here</a>.</p>
+<p>If a quick search doesn?t turn up an existing JIRA issue for the work you want to contribute,
+create it. Please discuss your proposal with a committer or the component lead in JIRA or,
+alternatively, on the developer mailing list(<a href="mailto:dev@carbondata.apache.org">dev@carbondata.apache.org</a>).</p>
+<p>If there?s an existing JIRA issue for your intended contribution, please comment about your
+intended work. Once the work is understood, a committer will assign the issue to you.
+(If you don?t have a JIRA role yet, you?ll be added to the ?contributor? role.) If an issue is
+currently assigned, please check with the current assignee before reassigning.</p>
+<p>For moderate or large contributions, you should not start coding or writing a design doc unless
+there is a corresponding JIRA issue assigned to you for that work. Simple changes,
+like fixing typos, do not require an associated issue.</p>
+<h3>
+<a id="design" class="anchor" href="#design" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Design</h3>
+<p>To clearly express your thoughts and get early feedback from other community members, we encourage you to clearly scope, document the design of non-trivial contributions and discuss with the CarbonData community before you start coding.</p>
+<p>Generally, the JIRA issue is the best place to gather relevant design docs, comments, or references. It?s great to explicitly include relevant stakeholders early in the conversation. For designs that may be generally interesting, we also encourage conversations on the developer?s mailing list.</p>
+<h3>
+<a id="code" class="anchor" href="#code" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Code</h3>
+<p>We use GitHub?s pull request functionality to review proposed code changes.
+If you do not already have a personal GitHub account, sign up <a href="https://github.com" target=_blank>here</a>.</p>
+<h3>
+<a id="git-config" class="anchor" href="#git-config" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Git config</h3>
+<p>Ensure to finish the below config(user.email, user.name) before starting PR works.</p>
+<pre><code>$ git config --global user.email "you@example.com"
+$ git config --global user.name "Your Name"
+</code></pre>
+<h4>
+<a id="fork-the-repository-on-github" class="anchor" href="#fork-the-repository-on-github" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Fork the repository on GitHub</h4>
+<p>Go to the <a href="https://github.com/apache/carbondata" target=_blank>Apache CarbonData GitHub mirror</a> and
+fork the repository to your account.
+This will be your private workspace for staging changes.</p>
+<h4>
+<a id="clone-the-repository-locally" class="anchor" href="#clone-the-repository-locally" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Clone the repository locally</h4>
+<p>You are now ready to create the development environment on your local machine.
+Clone CarbonData?s read-only GitHub mirror.</p>
+<pre><code>$ git clone https://github.com/apache/carbondata.git
+$ cd carbondata
+</code></pre>
+<p>Add your forked repository as an additional Git remote, where you?ll push your changes.</p>
+<pre><code>$ git remote add &lt;GitHub_user&gt; https://github.com/&lt;GitHub_user&gt;/carbondata.git
+</code></pre>
+<p>You are now ready to start developing!</p>
+<h4>
+<a id="create-a-branch-in-your-fork" class="anchor" href="#create-a-branch-in-your-fork" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create a branch in your fork</h4>
+<p>You?ll work on your contribution in a branch in your own (forked) repository. Create a local branch,
+initialized with the state of the branch you expect your changes to be merged into.
+Keep in mind that we use several branches, including master, feature-specific, and
+release-specific branches. If you are unsure, initialize with the state of the master branch.</p>
+<pre><code>$ git fetch --all
+$ git checkout -b &lt;my-branch&gt; origin/master
+</code></pre>
+<p>At this point, you can start making and committing changes to this branch in a standard way.</p>
+<h4>
+<a id="syncing-and-pushing-your-branch" class="anchor" href="#syncing-and-pushing-your-branch" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Syncing and pushing your branch</h4>
+<p>Periodically while you work, and certainly before submitting a pull request, you should update
+your branch with the most recent changes to the target branch.</p>
+<pre><code>$ git pull --rebase
+</code></pre>
+<p>Remember to always use --rebase parameter to avoid extraneous merge commits.</p>
+<p>To push your local, committed changes to your (forked) repository on GitHub, run:</p>
+<pre><code>$ git push &lt;GitHub_user&gt; &lt;my-branch&gt;
+</code></pre>
+<h4>
+<a id="testing" class="anchor" href="#testing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Testing</h4>
+<p>All code should have appropriate unit testing coverage. New code should have new tests in the
+same contribution. Bug fixes should include a regression test to prevent the issue from reoccurring.</p>
+<p>For contributions to the Java code, run unit tests locally via Maven.</p>
+<pre><code>$ mvn clean verify
+</code></pre>
+<h3>
+<a id="review" class="anchor" href="#review" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Review</h3>
+<p>Once the initial code is complete and the tests pass, it?s time to start the code review process.
+We review and discuss all code, no matter who authors it. It?s a great way to build community,
+since you can learn from other developers, and they become familiar with your contribution.
+It also builds a strong project by encouraging a high quality bar and keeping code consistent
+throughout the project.</p>
+<h4>
+<a id="create-a-pull-request" class="anchor" href="#create-a-pull-request" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create a pull request</h4>
+<p>Organize your commits to make your reviewer?s job easier. Use the following command to
+re-order, squash, edit, or change description of individual commits.</p>
+<pre><code>$ git rebase -i origin/master
+</code></pre>
+<p>Navigate to the CarbonData GitHub mirror to create a pull request. The title of the pull request
+should be strictly in the following format:</p>
+<pre><code>[CARBONDATA-JiraTicketNumer][FeatureName] Description of pull request    
+</code></pre>
+<p>Please include a descriptive pull request message to help make the reviewer?s job easier:</p>
+<pre><code> - The root cause/problem statement
+ - What is the implemented solution
+</code></pre>
+<p>If you know a good committer to review your pull request, please make a comment like the following.
+If not, don?t worry, a committer will pick it up.</p>
+<pre><code>Hi @&lt;committer/reviewer name&gt;, can you please take a look?
+</code></pre>
+<h4>
+<a id="code-review-and-revision" class="anchor" href="#code-review-and-revision" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Code Review and Revision</h4>
+<p>During the code review process, don?t rebase your branch or otherwise modify published commits,
+since this can remove existing comment history and be confusing to the reviewer,
+When you make a revision, always push it in a new commit.</p>
+<p>Our GitHub mirror automatically provides pre-commit testing coverage using Jenkins.
+Please make sure those tests pass,the contribution cannot be merged otherwise.</p>
+<h4>
+<a id="lgtm" class="anchor" href="#lgtm" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LGTM</h4>
+<p>Once the reviewer is happy with the change, they?ll respond with an LGTM (?looks good to me!?).
+At this point, the committer will take over, possibly make some additional touch ups,
+and merge your changes into the codebase.</p>
+<p>In the case both the author and the reviewer are committers, either can merge the pull request.
+Just be sure to communicate clearly whose responsibility it is in this particular case.</p>
+<p>Thank you for your contribution to Apache CarbonData!</p>
+<h4>
+<a id="deleting-your-branchoptional" class="anchor" href="#deleting-your-branchoptional" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deleting your branch(optional)</h4>
+<p>Once the pull request is merged into the Apache CarbonData repository, you can safely delete the
+branch locally and purge it from your forked repository.</p>
+<p>From another local branch, run:</p>
+<pre><code>$ git fetch --all
+$ git branch -d &lt;my-branch&gt;
+$ git push &lt;GitHub_user&gt; --delete &lt;my-branch&gt;
+</code></pre>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__contri').addClass('selected'); });
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/index.html
----------------------------------------------------------------------
diff --git a/content/index.html b/content/index.html
index e4e0c0e..e9e6e3b 100644
--- a/content/index.html
+++ b/content/index.html
@@ -70,7 +70,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="">Documentation</a></li>
+                    <li><a href="documentation.html" class="">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button"
                            aria-haspopup="true" aria-expanded="false">Community <span

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/installation-guide.html
----------------------------------------------------------------------
diff --git a/content/installation-guide.html b/content/installation-guide.html
index 2e7fab6..696db00 100644
--- a/content/installation-guide.html
+++ b/content/installation-guide.html
@@ -452,4 +452,4 @@ hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/introduction.html
----------------------------------------------------------------------
diff --git a/content/introduction.html b/content/introduction.html
new file mode 100644
index 0000000..29d2aa1
--- /dev/null
+++ b/content/introduction.html
@@ -0,0 +1,361 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div><h2>
+<a id="what-is-carbondata" class="anchor" href="#what-is-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>What is CarbonData</h2>
+<p>CarbonData is a fully indexed columnar and Hadoop native data-store for processing heavy analytical workloads and detailed queries on big data with Spark SQL. CarbonData allows faster interactive queries over PetaBytes of data.</p>
+<h2>
+<a id="what-does-this-mean" class="anchor" href="#what-does-this-mean" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>What does this mean</h2>
+<p>CarbonData has specially engineered optimizations like multi level indexing, compression and encoding techniques targeted to improve performance of analytical queries which can include filters, aggregation and distinct counts where users expect sub second response time for queries on TB level data on commodity hardware clusters with just a few nodes.</p>
+<p>CarbonData has</p>
+<ul>
+<li>
+<p><strong>Unique data organisation</strong> for faster retrievals and minimise amount of data retrieved</p>
+</li>
+<li>
+<p><strong>Advanced push down optimisations</strong> for deep integration with Spark so as to improvise the Spark DataSource API and other experimental features thereby ensure computing is performed close to the data to minimise amount of data read, processed, converted and transmitted(shuffled)</p>
+</li>
+<li>
+<p><strong>Multi level indexing</strong> to efficiently prune the files and data to be scanned and hence reduce I/O scans and CPU processing</p>
+</li>
+</ul>
+<h2>
+<a id="architecture" class="anchor" href="#architecture" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Architecture</h2>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_architecture.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_architecture.png" alt="" style="max-width:100%;"></a></p>
+<h4>
+<a id="spark-interface-layer" class="anchor" href="#spark-interface-layer" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Spark Interface Layer:</h4>
+<p>CarbonData has deep integration with Apache Spark.CarbonData integrates custom Parser,Strategies,Optimization rules into Spark to take advantage of computing performed closer to data.</p>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_spark_integration.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_spark_integration.png" alt="" style="max-width:100%;"></a></p>
+<ol>
+<li>
+<strong>Carbon parser</strong> Enhances Spark?s SQL parser to support Carbon specific DDL and DML commands to create carbon table, create aggregate tables, manage data loading, data retention and cleanup.</li>
+<li>
+<strong>Carbon Strategies</strong>:- Modify Spark SQL?s physical query execution plan to push down possible operations to Carbon for example:- Grouping, Distinct Count, Top N etc.. for improving query performance.</li>
+<li>
+<strong>Carbon Data RDD</strong>:- Makes the data present in Carbon tables visible to Spark as a RDD which enables spark to perform distributed computation on Carbon tables.</li>
+</ol>
+<h4>
+<a id="carbon-processor" class="anchor" href="#carbon-processor" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Carbon Processor:</h4>
+<p>Receives a query execution fragment from spark and executes the same on the Carbon storage. This involves Scanning the carbon store files for matching record, using the indices to directly locate the row sets and even the rows that may containing the data being searched for. The Carbon processor also performs all pushed down operations such as</p>
+<p>Aggregation/Group By</p>
+<p>Distinct Count</p>
+<p>Top N</p>
+<p>Expression Evaluation</p>
+<p>And many more?</p>
+<h4>
+<a id="carbon-storage" class="anchor" href="#carbon-storage" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Carbon Storage:</h4>
+<p>Custom columnar data store which is heavily compressed, binary, dictionary encoded and heavily indexed.Usaually stored in HDFS.</p>
+<h2>
+<a id="carbondata-features" class="anchor" href="#carbondata-features" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Features</h2>
+<p>CarbonData has rich set of featues to support various use cases in Big Data analytics.</p>
+<h2>
+<a id="design" class="anchor" href="#design" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Design</h2>
+<ul>
+<li>
+<h3>
+<a id="dictionary-encoding" class="anchor" href="#dictionary-encoding" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Dictionary Encoding</h3>
+</li>
+</ul>
+<p>CarbonData supports encoding of data with suggogate values to reduce storage space and speed up processing.Most databases and big data SQL data stores adopt dictionary encoding(integer surrogate numbers) to achieve data compression.Unlike other column store databases where the dictionary is local to each data block, CarbonData maintains a global dictionary which provides opportunity for lazy conversion to actual values enabling all computation to be performed on the lightweight surrogate values.</p>
+<h5>
+<a id="dictionary-generation" class="anchor" href="#dictionary-generation" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Dictionary generation</h5>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_dict_encoding.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_dict_encoding.png" alt="" style="max-width:100%;"></a></p>
+<h5>
+<a id=.htmlk-indexing" class="anchor" href=".htmlk-indexing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>MDK Indexing</h5>
+<p>All the surrogate keys are byte packed to generate an MDK (Multi Dimensional Key) Index.</p>
+<p>Any non surrogate columns of String data types are compressed using one of the configured compression algorithms and stored.For those numeric columns where surrogates are not generated, such data is stored as it is after compression.</p>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk.png" alt="image-20180903212418381" style="max-width:100%;"></a></p>
+<h5>
+<a id="sorted.htmlk" class="anchor" href="#sorted.htmlk" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Sorted MDK</h5>
+<p>The data is sorted based on the MDK Index.Sorting helps for logical grouping of similar data and there by aids in faster look up during query.</p>
+<h4>
+<a id="" class="anchor" href="#" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk_sort.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk_sort.png" alt="image-20180903212525214" style="max-width:100%;"></a>
+</h4>
+<h5>
+<a id="custom-columnar-encoding" class="anchor" href="#custom-columnar-encoding" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Custom Columnar Encoding</h5>
+<p>The Sorted MDK Index is split into each column.Unlike other stores where the column is compressed and stored as it is, CarbonData sorts this column data so that Binary Search can be performed on individual column data based on the filter conditions.This aids in magnitude increase in query performance and also in better compression.Since the individual column's data gets sorted, it is necessary to maintain the row mapping with the sorted MDK Index data in order to retrieve data from other columns which are not participating in filter.This row mapping is termed as <strong>Inverted Index</strong> and is stored along with the column data.The below picture depicts the logical column view.User has the option to <strong>turn off</strong> Inverted Index for such columns where filters are never applied or is very rare.In such cases, scanning would be sequential, but can aid in reducing the storage size(occupied due to inverted index data).</p>
+<h4>
+<a id="-1" class="anchor" href="#-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_blocklet_view.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_blocklet_view.png" alt="" style="max-width:100%;"></a>
+</h4>
+<ul>
+<li>
+<h3>
+<a id="carbondata-storage-format" class="anchor" href="#carbondata-storage-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Storage Format</h3>
+<p>CarbonData has a unique storage structure which aids in efficient storage and retrieval of data.Please refer to <a href="#./file-structure-of-carbondata.html">File Structure of CarbonData</a> for detailed information on the format.</p>
+</li>
+<li>
+<h3>
+<a id="indexing" class="anchor" href="#indexing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Indexing</h3>
+<p>CarbonData maintains multiple indexes at multiple levels to assist in efficient pruning of unwanted data from scan during query.Also CarbonData has support for plugging in external indexing solutions to speed up the query process.</p>
+<h5>
+<a id="min-max-indexing" class="anchor" href="#min-max-indexing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Min-Max Indexing</h5>
+<p>Storing data along with index significantly accelerates query performance and reduces the I/O scans and CPU resources in case of filters in the query. CarbonData index consists of multiple levels of indices, a processing framework can leverage this index to reduce the number of tasks it needs to schedule and process. It can also do skip scan in more fine grained units (called blocklet) in task side scanning instead of scanning the whole file.  <strong>CarbonData maintains Min-Max Index for all the columns.</strong></p>
+<p>CarbonData maintains a separate index file which contains the footer information for efficient IO reads.</p>
+<p>Using the Min-Max info in these index files, two levels of filtering can be achieved.</p>
+<p>Min-Max at the carbondata file level,to efficiently prune the files when the filter condition doesn't fall in the range.This information when maintained at the Spark Driver, will help to efficiently schedule the tasks for scanning</p>
+<p>Min-Max at the blocklet level, to efficiently prune the blocklets when the filter condition doesn't fall in the range.This information when maintained at the executor can significantly reduce the amount unnecessary data processed by the executor tasks.</p>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-minmax-blocklet.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-minmax-blocklet.png" alt="" style="max-width:100%;"></a></p>
+</li>
+<li>
+<h4>
+<a id="datamaps" class="anchor" href="#datamaps" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DataMaps</h4>
+<p>DataMap is a framework for indexing and also for statistics that can be used to add primary index (Blocklet Index) , secondary index type and statistical type to CarbonData.</p>
+<p>DataMap is a standardized general interface which CarbonData uses to prune data blocks for scanning.</p>
+<p>DataMaps are of 2 types:</p>
+<p><strong>CG(Coarse Grained) DataMaps</strong> Can prune data to the blocklet or to Page level.ie., Holds information for deciding which blocks/blocklets to be scanned.This DataMap is used in Spark Driver to decide the number of tasks to be scheduled.</p>
+<p><strong>FG(Fine Grained) DataMaps</strong> Can prune data to row level.This DataMap is used in Spark executor for scanning an fetching the data much faster.</p>
+<p>Since DataMap interfaces are generalised, We can write a thin adaptor called as <strong>DataMap Providers</strong> to interface between CarbonData and other external Indexing engines. For eg., Lucene, Solr,ES,...</p>
+<p>CarbonData has its own DSL to create and manage DataMaps.Please refer to <a href="#./datamap/datamap-management.html#overview">CarbonData DSL</a> for more information.</p>
+<p>The below diagram explains about the DataMap execution in CarbonData.</p>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-datamap.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-datamap.png" alt="" style="max-width:100%;"></a></p>
+</li>
+<li>
+<h4>
+<a id="update--delete" class="anchor" href="#update--delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Update &amp; Delete</h4>
+</li>
+</ul>
+<p>CarbonData supports Update and delete operations over big data.This functionality is not targetted for OLTP scenarios where high concurrent update/delete are required.Following are the assumptions considered when this feature is designed.</p>
+<ol>
+<li>Updates or Deletes are periodic and in Bulk</li>
+<li>Updates or Deletes are atomic</li>
+<li>Data is immediately visible</li>
+<li>Concurrent query to be allowed during an update or delete operation</li>
+<li>Single statement auto-commit support (not OLTP-style transaction)</li>
+</ol>
+<p>Since data stored in HDFS are immutable,data blocks cannot be updated in-place.Re-write of entire data block is not efficient for IO and also is a slow process.</p>
+<p>To over come these limitations, CarbonData adopts methodology of writing a delta file containing the rows to be deleted and another delta file containing the values to be updated with.During processing, These two delta files are merged with the main carbondata file and the correct result is returned for the query.</p>
+<p>The below diagram describes the process.</p>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_update_delete.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_update_delete.png" alt="" style="max-width:100%;"></a></p>
+<h2>
+<a id="integration-with-big-data-ecosystem" class="anchor" href="#integration-with-big-data-ecosystem" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Integration with Big Data ecosystem</h2>
+<p>Refer to Integration with <a href="#./quick-start-guide.html#spark">Spark</a>, <a href="#./quick-start-guide.html#presto">Presto</a> for detailed information on integrating CarbonData with these execution engines.</p>
+<h2>
+<a id="scenarios-where-carbondata-is-suitable" class="anchor" href="#scenarios-where-carbondata-is-suitable" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Scenarios where CarbonData is suitable</h2>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__intro').addClass('selected'); });
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/language-manual.html
----------------------------------------------------------------------
diff --git a/content/language-manual.html b/content/language-manual.html
new file mode 100644
index 0000000..54c29ac
--- /dev/null
+++ b/content/language-manual.html
@@ -0,0 +1,273 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="overview" class="anchor" href="#overview" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Overview</h1>
+<p>CarbonData has its own parser, in addition to Spark's SQL Parser, to parse and process certain Commands related to CarbonData table handling. You can interact with the SQL interface using the <a href="https://spark.apache.org/docs/latest/sql-programming-guide.html#running-the-spark-sql-cli" rel="nofollow">command-line</a> or over <a href="https://spark.apache.org/docs/latest/sql-programming-guide.html#running-the-thrift-jdbcodbc-server" rel="nofollow">JDBC/ODBC</a>.</p>
+<ul>
+<li><a href="./supported-data-types-in-carbondata.html">Data Types</a></li>
+<li>Data Definition Statements
+<ul>
+<li>
+<a href="./ddl-of-carbondata.html">DDL:</a><a href="./ddl-of-carbondata.html#create-table">Create</a>,<a href="./ddl-of-carbondata.html#drop-table">Drop</a>,<a href="./ddl-of-carbondata.html#partition">Partition</a>,<a href="./ddl-of-carbondata.html#bucketing">Bucketing</a>,<a href="./ddl-of-carbondata.html#alter-table">Alter</a>,<a href="./ddl-of-carbondata.html#create-table-as-select">CTAS</a>,<a href="./ddl-of-carbondata.html#create-external-table">External Table</a>
+</li>
+<li>Indexes</li>
+<li>
+<a href="./datamap-management.html">DataMaps</a>
+<ul>
+<li><a href="./bloomfilter-datamap-guide.html">Bloom</a></li>
+<li><a href="./lucene-datamap-guide.html">Lucene</a></li>
+<li><a href="./preaggregate-datamap-guide.html">Pre-Aggregate</a></li>
+<li><a href="./timeseries-datamap-guide.html">Time Series</a></li>
+</ul>
+</li>
+<li>Materialized Views (MV)</li>
+<li><a href="./streaming-guide.html">Streaming</a></li>
+</ul>
+</li>
+<li>Data Manipulation Statements
+<ul>
+<li>
+<a href="./dml-of-carbondata.html">DML:</a> <a href="./dml-of-carbondata.html#load-data">Load</a>, <a href="./ddl-of-carbondata.html#insert-overwrite">Insert</a>, <a href="./dml-of-carbondata.html#update">Update</a>, <a href="./dml-of-carbondata.html#delete">Delete</a>
+</li>
+<li><a href="./segment-management-on-carbondata.html">Segment Management</a></li>
+</ul>
+</li>
+<li><a href="./configuration-parameters.html">Configuration Properties</a></li>
+</ul>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/lucene-datamap-guide.html
----------------------------------------------------------------------
diff --git a/content/lucene-datamap-guide.html b/content/lucene-datamap-guide.html
index 7b1598e..038cb9d 100644
--- a/content/lucene-datamap-guide.html
+++ b/content/lucene-datamap-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -302,6 +353,17 @@ release, user can do as following:</p>
 <li>Create the lucene datamap again by <code>CREATE DATAMAP</code> command.
 Basically, user can manually trigger the operation by re-building the datamap.</li>
 </ol>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -317,4 +379,4 @@ Basically, user can manually trigger the operation by re-building the datamap.</
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/partition-guide.html
----------------------------------------------------------------------
diff --git a/content/partition-guide.html b/content/partition-guide.html
index a9a1f8c..4e6e6c0 100644
--- a/content/partition-guide.html
+++ b/content/partition-guide.html
@@ -339,4 +339,4 @@ SegmentDir/part-0-0_batchno0-0-1502703086921.carbondata
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/pdf/maven-pdf-plugin.pdf
----------------------------------------------------------------------
diff --git a/content/pdf/maven-pdf-plugin.pdf b/content/pdf/maven-pdf-plugin.pdf
index 72c0425..cb6de01 100644
Binary files a/content/pdf/maven-pdf-plugin.pdf and b/content/pdf/maven-pdf-plugin.pdf differ


[11/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/datamap-developer-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/datamap-developer-guide.html b/src/main/webapp/datamap-developer-guide.html
index 9ec7a91..073af96 100644
--- a/src/main/webapp/datamap-developer-guide.html
+++ b/src/main/webapp/datamap-developer-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -165,19 +216,31 @@
 <p>DataMap is a data structure that can be used to accelerate certain query of the table. Different DataMap can be implemented by developers.
 Currently, there are two 2 types of DataMap supported:</p>
 <ol>
-<li>IndexDataMap: DataMap that leveraging index to accelerate filter query</li>
-<li>MVDataMap: DataMap that leveraging Materialized View to accelerate olap style query, like SPJG query (select, predicate, join, groupby)</li>
+<li>IndexDataMap: DataMap that leverages index to accelerate filter query</li>
+<li>MVDataMap: DataMap that leverages Materialized View to accelerate olap style query, like SPJG query (select, predicate, join, groupby)</li>
 </ol>
 <h3>
 <a id="datamap-provider" class="anchor" href="#datamap-provider" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DataMap provider</h3>
 <p>When user issues <code>CREATE DATAMAP dm ON TABLE main USING 'provider'</code>, the corresponding DataMapProvider implementation will be created and initialized.
 Currently, the provider string can be:</p>
 <ol>
-<li>preaggregate: one type of MVDataMap that do pre-aggregate of single table</li>
-<li>timeseries: one type of MVDataMap that do pre-aggregate based on time dimension of the table</li>
+<li>preaggregate: A type of MVDataMap that do pre-aggregate of single table</li>
+<li>timeseries: A type of MVDataMap that do pre-aggregate based on time dimension of the table</li>
 <li>class name IndexDataMapFactory  implementation: Developer can implement new type of IndexDataMap by extending IndexDataMapFactory</li>
 </ol>
 <p>When user issues <code>DROP DATAMAP dm ON TABLE main</code>, the corresponding DataMapProvider interface will be called.</p>
+<p>Details about <a href="./datamap-management.html#datamap-management">DataMap Management</a> and supported <a href="./datamap-management.html#overview">DSL</a> are documented <a href="./datamap-management.html">here</a>.</p>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -193,4 +256,4 @@ Currently, the provider string can be:</p>
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/datamap-management.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/datamap-management.html b/src/main/webapp/datamap-management.html
index 50191af..8727594 100644
--- a/src/main/webapp/datamap-management.html
+++ b/src/main/webapp/datamap-management.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -161,6 +212,21 @@
                                     <div>
 <h1>
 <a id="carbondata-datamap-management" class="anchor" href="#carbondata-datamap-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData DataMap Management</h1>
+<ul>
+<li><a href="#overview">Overview</a></li>
+<li><a href="#datamap-management">DataMap Management</a></li>
+<li><a href="#automatic-refresh">Automatic Refresh</a></li>
+<li><a href="#manual-refresh">Manual Refresh</a></li>
+<li><a href="#datamap-catalog">DataMap Catalog</a></li>
+<li>
+<a href="#datamap-related-commands">DataMap Related Commands</a>
+<ul>
+<li><a href="#explain">Explain</a></li>
+<li><a href="#show-datamap">Show DataMap</a></li>
+<li><a href="#compaction-on-datamap">Compaction on DataMap</a></li>
+</ul>
+</li>
+</ul>
 <h2>
 <a id="overview" class="anchor" href="#overview" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Overview</h2>
 <p>DataMap can be created using following DDL</p>
@@ -192,7 +258,7 @@
 <tr>
 <td>timeseries</td>
 <td>time dimension rollup table</td>
-<td>event_time, xx_granularity, please refer to <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/timeseries-datamap-guide.html" target=_blank>Timeseries DataMap</a>
+<td>event_time, xx_granularity, please refer to <a href="./timeseries-datamap-guide.html">Timeseries DataMap</a>
 </td>
 <td>Automatic</td>
 </tr>
@@ -224,7 +290,6 @@
 <li>Manual Refresh: Create datamap with <code>WITH DEFERRED REBUILD</code> in the statement</li>
 </ol>
 <p><strong>CAUTION:</strong>
-Manual refresh currently only works fine for MV, it has some bugs with other types of datamap in Carbondata 1.4.1, so we block this option for them in this version.
 If user create MV datamap without specifying <code>WITH DEFERRED REBUILD</code>, carbondata will give a warning and treat the datamap as deferred rebuild.</p>
 <h3>
 <a id="automatic-refresh" class="anchor" href="#automatic-refresh" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Automatic Refresh</h3>
@@ -293,6 +358,17 @@ Scan Table: default.datamap1_table
 <p>This feature applies for preaggregate datamap only</p>
 <p>Running Compaction command (<code>ALTER TABLE COMPACT</code>) on main table will <strong>not automatically</strong> compact the pre-aggregate tables created on the main table. User need to run Compaction command separately on each pre-aggregate table to compact them.</p>
 <p>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 tables. To further improve the query performance, compaction on pre-aggregate tables can be triggered to merge the segments and files in the pre-aggregate tables.</p>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -308,4 +384,4 @@ Scan Table: default.datamap1_table
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/ddl-of-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/ddl-of-carbondata.html b/src/main/webapp/ddl-of-carbondata.html
new file mode 100644
index 0000000..3e15a83
--- /dev/null
+++ b/src/main/webapp/ddl-of-carbondata.html
@@ -0,0 +1,1078 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="carbondata-data-definition-language" class="anchor" href="#carbondata-data-definition-language" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Data Definition Language</h1>
+<p>CarbonData DDL statements are documented here,which includes:</p>
+<ul>
+<li>
+<a href="#create-table">CREATE TABLE</a>
+<ul>
+<li><a href="#dictionary-encoding-configuration">Dictionary Encoding</a></li>
+<li><a href="#inverted-index-configuration">Inverted Index</a></li>
+<li><a href="#sort-columns-configuration">Sort Columns</a></li>
+<li><a href="#sort-scope-configuration">Sort Scope</a></li>
+<li><a href="#table-block-size-configuration">Table Block Size</a></li>
+<li><a href="#table-compaction-configuration">Table Compaction</a></li>
+<li><a href="#streaming">Streaming</a></li>
+<li><a href="#local-dictionary-configuration">Local Dictionary</a></li>
+<li><a href="#caching-minmax-value-for-required-columns">Caching Column Min/Max</a></li>
+<li><a href="#caching-at-block-or-blocklet-level">Caching Level</a></li>
+<li><a href="#support-flat-folder-same-as-hiveparquet">Hive/Parquet folder Structure</a></li>
+<li><a href="#string-longer-than-32000-characters">Extra Long String columns</a></li>
+</ul>
+</li>
+<li><a href="#create-table-as-select">CREATE TABLE AS SELECT</a></li>
+<li>
+<a href="#create-external-table">CREATE EXTERNAL TABLE</a>
+<ul>
+<li><a href="#create-external-table-on-managed-table-data-location">External Table on Transactional table location</a></li>
+<li><a href="#create-external-table-on-non-transactional-table-data-location">External Table on non-transactional table location</a></li>
+</ul>
+</li>
+<li><a href="#create-database">CREATE DATABASE</a></li>
+<li>
+<a href="#table-management">TABLE MANAGEMENT</a>
+<ul>
+<li><a href="#show-table">SHOW TABLE</a></li>
+<li>
+<a href="#alter-table">ALTER TABLE</a>
+<ul>
+<li><a href="#rename-table">RENAME TABLE</a></li>
+<li><a href="#add-columns">ADD COLUMNS</a></li>
+<li><a href="#drop-columns">DROP COLUMNS</a></li>
+<li><a href="#change-data-type">CHANGE DATA TYPE</a></li>
+<li><a href="#merge-index">MERGE INDEXES</a></li>
+<li><a href="#set-and-unset-for-local-dictionary-properties">SET/UNSET Local Dictionary Properties</a></li>
+</ul>
+</li>
+<li><a href="#drop-table">DROP TABLE</a></li>
+<li><a href="#refresh-table">REFRESH TABLE</a></li>
+<li><a href="#table-and-column-comment">COMMENTS</a></li>
+</ul>
+</li>
+<li>
+<a href="#partition">PARTITION</a>
+<ul>
+<li>
+<a href="#standard-partition">STANDARD PARTITION(HIVE)</a>
+<ul>
+<li><a href="#insert-overwrite">INSERT OVERWRITE PARTITION</a></li>
+</ul>
+</li>
+<li>
+<a href="#create-hash-partition-table">CARBONDATA PARTITION</a>
+<ul>
+<li><a href="#create-hash-partition-table">HASH PARTITION</a></li>
+<li><a href="#create-range-partition-table">RANGE PARTITION</a></li>
+<li><a href="#create-list-partition-table">LIST PARTITION</a></li>
+</ul>
+</li>
+<li><a href="#show-partitions">SHOW PARTITIONS</a></li>
+<li><a href="#add-a-new-partition">ADD PARTITION</a></li>
+<li><a href="#split-a-partition">SPLIT PARTITION</a></li>
+<li><a href="#drop-a-partition">DROP PARTITION</a></li>
+</ul>
+</li>
+<li><a href="#bucketing">BUCKETING</a></li>
+</ul>
+<h2>
+<a id="create-table" class="anchor" href="#create-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE</h2>
+<p>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.</p>
+<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type , ...)]
+STORED AS carbondata
+[TBLPROPERTIES (property_name=property_value, ...)]
+[LOCATION 'path']
+</code></pre>
+<p><strong>NOTE:</strong> CarbonData also supports "STORED AS carbondata" and "USING carbondata". Find example code at <a href="https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala" target=_blank>CarbonSessionExample</a> in the CarbonData repo.</p>
+<h3>
+<a id="usage-guidelines" class="anchor" href="#usage-guidelines" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
+<p><strong>Supported properties:</strong> <a href="#dictionary-encoding-configuration">DICTIONARY_INCLUDE</a>,<a href="#inverted-index-configuration">NO_INVERTED_INDEX</a>,<a href="#sort-columns-configuration">SORT_COLUMNS</a>,<a href="#sort-scope-configuration">SORT_SCOPE</a>,<a href="#table-block-size-configuration">TABLE_BLOCKSIZE</a>,<a href="#table-compaction-configuration">MAJOR_COMPACTION_SIZE</a>,</p>
+<p><a href="#table-compaction-configuration">AUTO_LOAD_MERGE</a>,<a href="#table-compaction-configuration">COMPACTION_LEVEL_THRESHOLD</a>,<a href="#table-compaction-configuration">COMPACTION_PRESERVE_SEGMENTS</a>,<a href="#table-compaction-configuration">ALLOWED_COMPACTION_DAYS</a>,</p>
+<p><a href="#streaming">streaming</a>,<a href="#local-dictionary-configuration">LOCAL_DICTIONARY_ENABLE</a>,<a href="#local-dictionary-configuration">LOCAL_DICTIONARY_THRESHOLD</a>,<a href="#local-dictionary-configuration">LOCAL_DICTIONARY_INCLUDE</a>,</p>
+<p><a href="#local-dictionary-configuration">LOCAL_DICTIONARY_EXCLUDE</a>,<a href="#caching-minmax-value-for-required-columns">COLUMN_META_CACHE</a>,<a href="#caching-at-block-or-blocklet-level">CACHE_LEVEL</a>,<a href="#support-flat-folder-same-as-hiveparquet">flat_folder</a>,<a href="#string-longer-than-32000-characters">LONG_STRING_COLUMNS</a>,<a href="#bucketing">BUCKETNUMBER</a>,</p>
+<p><a href="#bucketing">BUCKETCOLUMNS</a></p>
+<p>Following are the guidelines for TBLPROPERTIES, CarbonData's additional table options can be set via carbon.properties.</p>
+<ul>
+<li>
+<h5>
+<a id="dictionary-encoding-configuration" class="anchor" href="#dictionary-encoding-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Dictionary Encoding Configuration</h5>
+<p>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.</p>
+<pre><code>TBLPROPERTIES ('DICTIONARY_INCLUDE'='column1, column2')
+</code></pre>
+</li>
+</ul>
+<pre><code>```
+ NOTE: Dictionary Include/Exclude for complex child columns is not supported.
+</code></pre>
+<ul>
+<li>
+<h5>
+<a id="inverted-index-configuration" class="anchor" href="#inverted-index-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Inverted Index Configuration</h5>
+<p>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.</p>
+<pre><code>TBLPROPERTIES ('NO_INVERTED_INDEX'='column1, column3')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="sort-columns-configuration" class="anchor" href="#sort-columns-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Sort Columns Configuration</h5>
+<p>This property is for users to specify which columns belong to the MDK(Multi-Dimensions-Key) index.</p>
+<ul>
+<li>If users don't specify "SORT_COLUMN" property, by default MDK index be built by using all dimension columns except complex data type column.</li>
+<li>If this property is specified but with empty argument, then the table will be loaded without sort.</li>
+<li>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.</li>
+</ul>
+<pre><code>TBLPROPERTIES ('SORT_COLUMNS'='column1, column3')
+OR
+TBLPROPERTIES ('SORT_COLUMNS'='')
+</code></pre>
+<p>NOTE: Sort_Columns for Complex datatype columns is not supported.</p>
+</li>
+<li>
+<h5>
+<a id="sort-scope-configuration" class="anchor" href="#sort-scope-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Sort Scope Configuration</h5>
+<p>This property is for users to specify the scope of the sort during data load, following are the types of sort scope.</p>
+<ul>
+<li>LOCAL_SORT: It is the default sort scope.</li>
+<li>NO_SORT: It will load the data in unsorted manner, it will significantly increase load performance.</li>
+<li>BATCH_SORT: It increases the load performance but decreases the query performance if identified blocks &gt; parallelism.</li>
+<li>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.</li>
+</ul>
+</li>
+</ul>
+<pre><code>### Example:
+</code></pre>
+<pre><code> 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 ('SORT_COLUMNS'='productName,storeCity',
+                'SORT_SCOPE'='NO_SORT')
+</code></pre>
+<p><strong>NOTE:</strong> CarbonData also supports "using carbondata". Find example code at <a href="https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala" target=_blank>SparkSessionExample</a> in the CarbonData repo.</p>
+<ul>
+<li>
+<h5>
+<a id="table-block-size-configuration" class="anchor" href="#table-block-size-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Table Block Size Configuration</h5>
+<p>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.</p>
+<pre><code>TBLPROPERTIES ('TABLE_BLOCKSIZE'='512')
+</code></pre>
+<p><strong>NOTE:</strong> 512 or 512M both are accepted.</p>
+</li>
+<li>
+<h5>
+<a id="table-compaction-configuration" class="anchor" href="#table-compaction-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Table Compaction Configuration</h5>
+<p>These properties are table level compaction configurations, if not specified, system level configurations in carbon.properties will be used.
+Following are 5 configurations:</p>
+<ul>
+<li>MAJOR_COMPACTION_SIZE: same meaning as carbon.major.compaction.size, size in MB.</li>
+<li>AUTO_LOAD_MERGE: same meaning as carbon.enable.auto.load.merge.</li>
+<li>COMPACTION_LEVEL_THRESHOLD: same meaning as carbon.compaction.level.threshold.</li>
+<li>COMPACTION_PRESERVE_SEGMENTS: same meaning as carbon.numberof.preserve.segments.</li>
+<li>ALLOWED_COMPACTION_DAYS: same meaning as carbon.allowed.compaction.days.</li>
+</ul>
+<pre><code>TBLPROPERTIES ('MAJOR_COMPACTION_SIZE'='2048',
+               'AUTO_LOAD_MERGE'='true',
+               'COMPACTION_LEVEL_THRESHOLD'='5,6',
+               'COMPACTION_PRESERVE_SEGMENTS'='10',
+               'ALLOWED_COMPACTION_DAYS'='5')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="streaming" class="anchor" href="#streaming" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Streaming</h5>
+<p>CarbonData supports streaming ingestion for real-time data. You can create the ?streaming? table using the following table properties.</p>
+<pre><code>TBLPROPERTIES ('streaming'='true')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="local-dictionary-configuration" class="anchor" href="#local-dictionary-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Local Dictionary Configuration</h5>
+</li>
+</ul>
+<p>Columns for which dictionary is not generated needs more storage space and in turn more IO. Also since more data will have to be read during query, query performance also would suffer.Generating dictionary per blocklet for such columns would help in saving storage space and assist in improving query performance as carbondata is optimized for handling dictionary encoded columns more effectively.Generating dictionary internally per blocklet is termed as local dictionary. Please refer to <a href="./file-structure-of-carbondata.html">File structure of Carbondata</a> for understanding about the file structure of carbondata and meaning of terms like blocklet.</p>
+<p>Local Dictionary helps in:</p>
+<ol>
+<li>Getting more compression.</li>
+<li>Filter queries and full scan queries will be faster as filter will be done on encoded data.</li>
+<li>Reducing the store size and memory footprint as only unique values will be stored as part of local dictionary and corresponding data will be stored as encoded data.</li>
+<li>Getting higher IO throughput.</li>
+</ol>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>
+<p>Following Data Types are Supported for Local Dictionary:</p>
+<ul>
+<li>STRING</li>
+<li>VARCHAR</li>
+<li>CHAR</li>
+</ul>
+</li>
+<li>
+<p>Following Data Types are not Supported for Local Dictionary:</p>
+<ul>
+<li>SMALLINT</li>
+<li>INTEGER</li>
+<li>BIGINT</li>
+<li>DOUBLE</li>
+<li>DECIMAL</li>
+<li>TIMESTAMP</li>
+<li>DATE</li>
+<li>BOOLEAN</li>
+</ul>
+</li>
+<li>
+<p>In case of multi-level complex dataType columns, primitive string/varchar/char columns are considered for local dictionary generation.</p>
+</li>
+</ul>
+<p>Local dictionary will have to be enabled explicitly during create table or by enabling the <strong>system property</strong> <em><strong>carbon.local.dictionary.enable</strong></em>. By default, Local Dictionary will be disabled for the carbondata table.</p>
+<p>Local Dictionary can be configured using the following properties during create table command:</p>
+<table>
+<thead>
+<tr>
+<th>Properties</th>
+<th>Default value</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>LOCAL_DICTIONARY_ENABLE</td>
+<td>false</td>
+<td>Whether to enable local dictionary generation. <strong>NOTE:</strong> If this property is defined, it will override the value configured at system level by 'carbon.local.dictionary.enable'</td>
+</tr>
+<tr>
+<td>LOCAL_DICTIONARY_THRESHOLD</td>
+<td>10000</td>
+<td>The maximum cardinality of a column upto which carbondata can try to generate local dictionary (maximum - 100000)</td>
+</tr>
+<tr>
+<td>LOCAL_DICTIONARY_INCLUDE</td>
+<td>string/varchar/char columns</td>
+<td>Columns for which Local Dictionary has to be generated.<strong>NOTE:</strong> Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.</td>
+</tr>
+<tr>
+<td>LOCAL_DICTIONARY_EXCLUDE</td>
+<td>none</td>
+<td>Columns for which Local Dictionary need not be generated.</td>
+</tr>
+</tbody>
+</table>
+<p><strong>Fallback behavior:</strong></p>
+<ul>
+<li>When the cardinality of a column exceeds the threshold, it triggers a fallback and the generated dictionary will be reverted and data loading will be continued without dictionary encoding.</li>
+</ul>
+<p><strong>NOTE:</strong> When fallback is triggered, the data loading performance will decrease as encoded data will be discarded and the actual data is written to the temporary sort files.</p>
+<p><strong>Points to be noted:</strong></p>
+<ol>
+<li>
+<p>Reduce Block size:</p>
+<p>Number of Blocks generated is less in case of Local Dictionary as compression ratio is high. This may reduce the number of tasks launched during query, resulting in degradation of query performance if the pruned blocks are less compared to the number of parallel tasks which can be run. So it is recommended to configure smaller block size which in turn generates more number of blocks.</p>
+</li>
+<li>
+<p>All the page-level data for a blocklet needs to be maintained in memory until all the pages encoded for local dictionary is processed in order to handle fallback. Hence the memory required for local dictionary based table is more and this memory increase is proportional to number of columns.</p>
+</li>
+</ol>
+<h3>
+<a id="example" class="anchor" href="#example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
+<pre><code>CREATE TABLE carbontable(
+          
+            column1 string,
+          
+            column2 string,
+          
+            column3 LONG )
+          
+  STORED BY 'carbondata'
+  TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='true','LOCAL_DICTIONARY_THRESHOLD'='1000',
+  'LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>We recommend to use Local Dictionary when cardinality is high but is distributed across multiple loads</li>
+<li>On a large cluster, decoding data can become a bottleneck for global dictionary as there will be many remote reads. In this scenario, it is better to use Local Dictionary.</li>
+<li>When cardinality is less, but loads are repetitive, it is better to use global dictionary as local dictionary generates multiple dictionary files at blocklet level increasing redundancy.</li>
+</ul>
+<ul>
+<li>
+<h5>
+<a id="caching-minmax-value-for-required-columns" class="anchor" href="#caching-minmax-value-for-required-columns" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Caching Min/Max Value for Required Columns</h5>
+<p>By default, CarbonData caches min and max values of all the columns in schema.  As the load increases, the memory required to hold the min and max values increases considerably. This feature enables you to configure min and max values only for the required columns, resulting in optimized memory usage.</p>
+<p>Following are the valid values for COLUMN_META_CACHE:</p>
+<ul>
+<li>If you want no column min/max values to be cached in the driver.</li>
+</ul>
+<pre><code>COLUMN_META_CACHE=??
+</code></pre>
+<ul>
+<li>If you want only col1 min/max values to be cached in the driver.</li>
+</ul>
+<pre><code>COLUMN_META_CACHE=?col1?
+</code></pre>
+<ul>
+<li>If you want min/max values to be cached in driver for all the specified columns.</li>
+</ul>
+<pre><code>COLUMN_META_CACHE=?col1,col2,col3,??
+</code></pre>
+<p>Columns to be cached can be specified either while creating table or after creation of the table.
+During create table operation; specify the columns to be cached in table properties.</p>
+<p>Syntax:</p>
+<pre><code>CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,?) STORED BY ?carbondata? TBLPROPERTIES (?COLUMN_META_CACHE?=?col1,col2,??)
+</code></pre>
+<p>Example:</p>
+<pre><code>CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES (?COLUMN_META_CACHE?=?name?)
+</code></pre>
+<p>After creation of table or on already created tables use the alter table command to configure the columns to be cached.</p>
+<p>Syntax:</p>
+<pre><code>ALTER TABLE [dbName].tableName SET TBLPROPERTIES (?COLUMN_META_CACHE?=?col1,col2,??)
+</code></pre>
+<p>Example:</p>
+<pre><code>ALTER TABLE employee SET TBLPROPERTIES (?COLUMN_META_CACHE?=?city?)
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="caching-at-block-or-blocklet-level" class="anchor" href="#caching-at-block-or-blocklet-level" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Caching at Block or Blocklet Level</h5>
+<p>This feature allows you to maintain the cache at Block level, resulting in optimized usage of the memory. The memory consumption is high if the Blocklet level caching is maintained as a Block can have multiple Blocklet.</p>
+<p>Following are the valid values for CACHE_LEVEL:</p>
+<p><em>Configuration for caching in driver at Block level (default value).</em></p>
+<pre><code>CACHE_LEVEL= ?BLOCK?
+</code></pre>
+<p><em>Configuration for caching in driver at Blocklet level.</em></p>
+<pre><code>CACHE_LEVEL= ?BLOCKLET?
+</code></pre>
+<p>Cache level can be specified either while creating table or after creation of the table.
+During create table operation specify the cache level in table properties.</p>
+<p>Syntax:</p>
+<pre><code>CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,?) STORED BY ?carbondata? TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
+</code></pre>
+<p>Example:</p>
+<pre><code>CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
+</code></pre>
+<p>After creation of table or on already created tables use the alter table command to configure the cache level.</p>
+<p>Syntax:</p>
+<pre><code>ALTER TABLE [dbName].tableName SET TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
+</code></pre>
+<p>Example:</p>
+<pre><code>ALTER TABLE employee SET TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="support-flat-folder-same-as-hiveparquet" class="anchor" href="#support-flat-folder-same-as-hiveparquet" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Support Flat folder same as Hive/Parquet</h5>
+<p>This feature allows all carbondata and index files to keep directy under tablepath. Currently all carbondata/carbonindex files written under tablepath/Fact/Part0/Segment_NUM folder and it is not same as hive/parquet folder structure. This feature makes all files written will be directly under tablepath, it does not maintain any segment folder structure.This is useful for interoperability between the execution engines and plugin with other execution engines like hive or presto becomes easier.</p>
+<p>Following table property enables this feature and default value is false.</p>
+<pre><code> 'flat_folder'='true'
+</code></pre>
+<p>Example:</p>
+<pre><code>CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES ('flat_folder'='true')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="string-longer-than-32000-characters" class="anchor" href="#string-longer-than-32000-characters" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>String longer than 32000 characters</h5>
+<p>In common scenarios, the length of string is less than 32000,
+so carbondata stores the length of content using Short to reduce memory and space consumption.
+To support string longer than 32000 characters, carbondata introduces a table property called <code>LONG_STRING_COLUMNS</code>.
+For these columns, carbondata internally stores the length of content using Integer.</p>
+<p>You can specify the columns as 'long string column' using below tblProperties:</p>
+<pre><code>// specify col1, col2 as long string columns
+TBLPROPERTIES ('LONG_STRING_COLUMNS'='col1,col2')
+</code></pre>
+<p>Besides, you can also use this property through DataFrame by</p>
+<pre><code>df.format("carbondata")
+  .option("tableName", "carbonTable")
+  .option("long_string_columns", "col1, col2")
+  .save()
+</code></pre>
+<p>If you are using Carbon-SDK, you can specify the datatype of long string column as <code>varchar</code>.
+You can refer to SDKwriterTestCase for example.</p>
+<p><strong>NOTE:</strong> The LONG_STRING_COLUMNS can only be string/char/varchar columns and cannot be dictionary_include/sort_columns/complex columns.</p>
+</li>
+</ul>
+<h2>
+<a id="create-table-as-select" class="anchor" href="#create-table-as-select" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE AS SELECT</h2>
+<p>This function allows user 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.</p>
+<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name 
+STORED BY 'carbondata' 
+[TBLPROPERTIES (key1=val1, key2=val2, ...)] 
+AS select_statement;
+</code></pre>
+<h3>
+<a id="examples" class="anchor" href="#examples" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples</h3>
+<pre><code>carbon.sql("CREATE TABLE source_table(
+                           id INT,
+                           name STRING,
+                           city STRING,
+                           age INT)
+            STORED AS parquet")
+carbon.sql("INSERT INTO source_table SELECT 1,'bob','shenzhen',27")
+carbon.sql("INSERT INTO source_table SELECT 2,'david','shenzhen',31")
+
+carbon.sql("CREATE TABLE target_table
+            STORED BY 'carbondata'
+            AS SELECT city,avg(age) FROM source_table GROUP BY city")
+            
+carbon.sql("SELECT * FROM target_table").show
+  // results:
+  //    +--------+--------+
+  //    |    city|avg(age)|
+  //    +--------+--------+
+  //    |shenzhen|    29.0|
+  //    +--------+--------+
+
+</code></pre>
+<h2>
+<a id="create-external-table" class="anchor" href="#create-external-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE EXTERNAL TABLE</h2>
+<p>This function allows user to create external table by specifying location.</p>
+<pre><code>CREATE EXTERNAL TABLE [IF NOT EXISTS] [db_name.]table_name 
+STORED BY 'carbondata' LOCATION ?$FilesPath?
+</code></pre>
+<h3>
+<a id="create-external-table-on-managed-table-data-location" class="anchor" href="#create-external-table-on-managed-table-data-location" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create external table on managed table data location.</h3>
+<p>Managed table data location provided will have both FACT and Metadata folder.
+This data can be generated by creating a normal carbon table and use this path as $FilesPath in the above syntax.</p>
+<p><strong>Example:</strong></p>
+<pre><code>sql("CREATE TABLE origin(key INT, value STRING) STORED BY 'carbondata'")
+sql("INSERT INTO origin select 100,'spark'")
+sql("INSERT INTO origin select 200,'hive'")
+// creates a table in $storeLocation/origin
+
+sql(s"""
+|CREATE EXTERNAL TABLE source
+|STORED BY 'carbondata'
+|LOCATION '$storeLocation/origin'
+""".stripMargin)
+checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin"))
+</code></pre>
+<h3>
+<a id="create-external-table-on-non-transactional-table-data-location" class="anchor" href="#create-external-table-on-non-transactional-table-data-location" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create external table on Non-Transactional table data location.</h3>
+<p>Non-Transactional table data location will have only carbondata and carbonindex files, there will not be a metadata folder (table status and schema).
+Our SDK module currently support writing data in this format.</p>
+<p><strong>Example:</strong></p>
+<pre><code>sql(
+s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbondata' LOCATION
+|'$writerPath' """.stripMargin)
+</code></pre>
+<p>Here writer path will have carbondata and index files.
+This can be SDK output. Refer <a href="./sdk-guide.html">SDK Guide</a>.</p>
+<p><strong>Note:</strong></p>
+<ol>
+<li>Dropping of the external table should not delete the files present in the location.</li>
+<li>When external table is created on non-transactional table data,
+external table will be registered with the schema of carbondata files.
+If multiple files with different schema is present, exception will be thrown.
+So, If table registered with one schema and files are of different schema,
+suggest to drop the external table and create again to register table with new schema.</li>
+</ol>
+<h2>
+<a id="create-database" class="anchor" href="#create-database" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE DATABASE</h2>
+<p>This function creates a new database. By default the database is created in Carbon store location, but you can also specify custom location.</p>
+<pre><code>CREATE DATABASE [IF NOT EXISTS] database_name [LOCATION path];
+</code></pre>
+<h3>
+<a id="example-1" class="anchor" href="#example-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example</h3>
+<pre><code>CREATE DATABASE carbon LOCATION ?hdfs://name_cluster/dir1/carbonstore?;
+</code></pre>
+<h2>
+<a id="table-management" class="anchor" href="#table-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>TABLE MANAGEMENT</h2>
+<h3>
+<a id="show-table" class="anchor" href="#show-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW TABLE</h3>
+<p>This command can be used to list all the tables in current database or all the tables of a specific database.</p>
+<pre><code>SHOW TABLES [IN db_Name]
+</code></pre>
+<p>Example:</p>
+<pre><code>SHOW TABLES
+OR
+SHOW TABLES IN defaultdb
+</code></pre>
+<h3>
+<a id="alter-table" class="anchor" href="#alter-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>ALTER TABLE</h3>
+<p>The following section introduce the commands to modify the physical or logical state of the existing table(s).</p>
+<ul>
+<li>
+<h5>
+<a id="rename-table" class="anchor" href="#rename-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>RENAME TABLE</h5>
+<p>This command is used to rename the existing table.</p>
+<pre><code>ALTER TABLE [db_name.]table_name RENAME TO new_table_name
+</code></pre>
+<p>Examples:</p>
+<pre><code>ALTER TABLE carbon RENAME TO carbonTable
+OR
+ALTER TABLE test_db.carbon RENAME TO test_db.carbonTable
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="add-columns" class="anchor" href="#add-columns" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>ADD COLUMNS</h5>
+<p>This command is used to add a new column to the existing table.</p>
+<pre><code>ALTER TABLE [db_name.]table_name ADD COLUMNS (col_name data_type,...)
+TBLPROPERTIES('DICTIONARY_INCLUDE'='col_name,...',
+'DEFAULT.VALUE.COLUMN_NAME'='default_value')
+</code></pre>
+<p>Examples:</p>
+<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING)
+</code></pre>
+<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DICTIONARY_INCLUDE'='a1')
+</code></pre>
+<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DEFAULT.VALUE.a1'='10')
+</code></pre>
+<p>NOTE: Add Complex datatype columns is not supported.</p>
+</li>
+</ul>
+<p>Users can specify which columns to include and exclude for local dictionary generation after adding new columns. These will be appended with the already existing local dictionary include and exclude columns of main table respectively.</p>
+<pre><code>   ALTER TABLE carbon ADD COLUMNS (a1 STRING, b1 STRING) TBLPROPERTIES('LOCAL_DICTIONARY_INCLUDE'='a1','LOCAL_DICTIONARY_EXCLUDE'='b1')
+</code></pre>
+<ul>
+<li>
+<h5>
+<a id="drop-columns" class="anchor" href="#drop-columns" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DROP COLUMNS</h5>
+<p>This command is used to delete the existing column(s) in a table.</p>
+<pre><code>ALTER TABLE [db_name.]table_name DROP COLUMNS (col_name, ...)
+</code></pre>
+<p>Examples:</p>
+<pre><code>ALTER TABLE carbon DROP COLUMNS (b1)
+OR
+ALTER TABLE test_db.carbon DROP COLUMNS (b1)
+
+ALTER TABLE carbon DROP COLUMNS (c1,d1)
+</code></pre>
+<p>NOTE: Drop Complex child column is not supported.</p>
+</li>
+<li>
+<h5>
+<a id="change-data-type" class="anchor" href="#change-data-type" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CHANGE DATA TYPE</h5>
+<p>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.</p>
+<pre><code>ALTER TABLE [db_name.]table_name CHANGE col_name col_name changed_column_type
+</code></pre>
+<p>Valid Scenarios</p>
+<ul>
+<li>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.</li>
+<li>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.</li>
+<li>
+<strong>NOTE:</strong> The allowed range is 38,38 (precision, scale) and is a valid upper case scenario which is not resulting in data loss.</li>
+</ul>
+<p>Example1:Changing data type of column a1 from INT to BIGINT.</p>
+<pre><code>ALTER TABLE test_db.carbon CHANGE a1 a1 BIGINT
+</code></pre>
+<p>Example2:Changing decimal precision of column a1 from 10 to 18.</p>
+<pre><code>ALTER TABLE test_db.carbon CHANGE a1 a1 DECIMAL(18,2)
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="merge-index" class="anchor" href="#merge-index" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>MERGE INDEX</h5>
+<p>This command is used to merge all the CarbonData index files (.carbonindex) inside a segment to a single CarbonData index merge file (.carbonindexmerge). This enhances the first query performance.</p>
+<pre><code> ALTER TABLE [db_name.]table_name COMPACT 'SEGMENT_INDEX'
+</code></pre>
+<pre><code>Examples:
+```
+ALTER TABLE test_db.carbon COMPACT 'SEGMENT_INDEX'
+```
+**NOTE:**
+
+* Merge index is not supported on streaming table.
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="set-and-unset-for-local-dictionary-properties" class="anchor" href="#set-and-unset-for-local-dictionary-properties" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SET and UNSET for Local Dictionary Properties</h5>
+<p>When set command is used, all the newly set properties will override the corresponding old properties if exists.</p>
+<p>Example to SET Local Dictionary Properties:</p>
+<pre><code>ALTER TABLE tablename SET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='false','LOCAL_DICTIONARY_THRESHOLD'='1000','LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
+</code></pre>
+<p>When Local Dictionary properties are unset, corresponding default values will be used for these properties.</p>
+<p>Example to UNSET Local Dictionary Properties:</p>
+<pre><code>ALTER TABLE tablename UNSET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE','LOCAL_DICTIONARY_THRESHOLD','LOCAL_DICTIONARY_INCLUDE','LOCAL_DICTIONARY_EXCLUDE')
+</code></pre>
+<p><strong>NOTE:</strong> For old tables, by default, local dictionary is disabled. If user wants local dictionary for these tables, user can enable/disable local dictionary for new data at their discretion.
+This can be achieved by using the alter table set command.</p>
+</li>
+</ul>
+<h3>
+<a id="drop-table" class="anchor" href="#drop-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DROP TABLE</h3>
+<p>This command is used to delete an existing table.</p>
+<pre><code>DROP TABLE [IF EXISTS] [db_name.]table_name
+</code></pre>
+<p>Example:</p>
+<pre><code>DROP TABLE IF EXISTS productSchema.productSalesTable
+</code></pre>
+<h3>
+<a id="refresh-table" class="anchor" href="#refresh-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>REFRESH TABLE</h3>
+<p>This command is used to register Carbon table to HIVE meta store catalogue from existing Carbon table data.</p>
+<pre><code>REFRESH TABLE $db_NAME.$table_NAME
+</code></pre>
+<p>Example:</p>
+<pre><code>REFRESH TABLE dbcarbon.productSalesTable
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>The new database name and the old database name should be same.</li>
+<li>Before executing this command the old table schema and data should be copied into the new database location.</li>
+<li>If the table is aggregate table, then all the aggregate tables should be copied to the new database location.</li>
+<li>For old store, the time zone of the source and destination cluster should be same.</li>
+<li>If old cluster used HIVE meta store to store schema, refresh will not work as schema file does not exist in file system.</li>
+</ul>
+<h3>
+<a id="table-and-column-comment" class="anchor" href="#table-and-column-comment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Table and Column Comment</h3>
+<p>You can provide more information on table by using table comment. Similarly you can provide more information about a particular column using column comment.
+You can see the column comment of an existing table using describe formatted command.</p>
+<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type [COMMENT col_comment], ...)]
+  [COMMENT table_comment]
+STORED BY 'carbondata'
+[TBLPROPERTIES (property_name=property_value, ...)]
+</code></pre>
+<p>Example:</p>
+<pre><code>CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
+                              productNumber Int COMMENT 'unique serial number for product')
+COMMENT ?This is table comment?
+ STORED BY 'carbondata'
+ TBLPROPERTIES ('DICTIONARY_INCLUDE'='productNumber')
+</code></pre>
+<p>You can also SET and UNSET table comment using ALTER command.</p>
+<p>Example to SET table comment:</p>
+<pre><code>ALTER TABLE carbon SET TBLPROPERTIES ('comment'='this table comment is modified');
+</code></pre>
+<p>Example to UNSET table comment:</p>
+<pre><code>ALTER TABLE carbon UNSET TBLPROPERTIES ('comment');
+</code></pre>
+<h2>
+<a id="partition" class="anchor" href="#partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>PARTITION</h2>
+<h3>
+<a id="standard-partition" class="anchor" href="#standard-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>STANDARD PARTITION</h3>
+<p>The partition is similar as spark and hive partition, user can use any column to build partition:</p>
+<h4>
+<a id="create-partition-table" class="anchor" href="#create-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Partition Table</h4>
+<p>This command allows you to create table with partition.</p>
+<pre><code>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, ...)]
+</code></pre>
+<p>Example:</p>
+<pre><code> 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'
+</code></pre>
+<p>NOTE: Hive partition is not supported on complex datatype columns.</p>
+<h4>
+<a id="show-partitions" class="anchor" href="#show-partitions" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Show Partitions</h4>
+<p>This command gets the Hive partition information of the table</p>
+<pre><code>SHOW PARTITIONS [db_name.]table_name
+</code></pre>
+<h4>
+<a id="drop-partition" class="anchor" href="#drop-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Drop Partition</h4>
+<p>This command drops the specified Hive partition only.</p>
+<pre><code>ALTER TABLE table_name DROP [IF EXISTS] PARTITION (part_spec, ...)
+</code></pre>
+<p>Example:</p>
+<pre><code>ALTER TABLE locationTable DROP PARTITION (country = 'US');
+</code></pre>
+<h4>
+<a id="insert-overwrite" class="anchor" href="#insert-overwrite" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Insert OVERWRITE</h4>
+<p>This command allows you to insert or load overwrite on a specific partition.</p>
+<pre><code> INSERT OVERWRITE TABLE table_name
+ PARTITION (column = 'partition_name')
+ select_statement
+</code></pre>
+<p>Example:</p>
+<pre><code>INSERT OVERWRITE TABLE partitioned_user
+PARTITION (country = 'US')
+SELECT * FROM another_user au 
+WHERE au.country = 'US';
+</code></pre>
+<h3>
+<a id="carbondata-partitionhashrangelist----alpha-feature-this-partition-feature-does-not-support-update-and-delete-data" class="anchor" href="#carbondata-partitionhashrangelist----alpha-feature-this-partition-feature-does-not-support-update-and-delete-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CARBONDATA PARTITION(HASH,RANGE,LIST) -- Alpha feature, this partition feature does not support update and delete data.</h3>
+<p>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.</p>
+<h3>
+<a id="create-hash-partition-table" class="anchor" href="#create-hash-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Hash Partition Table</h3>
+<p>This command allows us to create hash partition.</p>
+<pre><code>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' ...)]
+</code></pre>
+<p><strong>NOTE:</strong> N is the number of hash partitions</p>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<h3>
+<a id="create-range-partition-table" class="anchor" href="#create-range-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Range Partition Table</h3>
+<p>This command allows us to create range partition.</p>
+<pre><code>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, ...')]
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>The 'RANGE_INFO' must be defined in ascending order in the table properties.</li>
+<li>The default format for partition column of Date/Timestamp type is yyyy-MM-dd. Alternate formats for Date/Timestamp could be defined in CarbonProperties.</li>
+</ul>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<h3>
+<a id="create-list-partition-table" class="anchor" href="#create-list-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create List Partition Table</h3>
+<p>This command allows us to create list partition.</p>
+<pre><code>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, ...')]
+</code></pre>
+<p><strong>NOTE:</strong> List partition supports list info in one level group.</p>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<h3>
+<a id="show-partitions-1" class="anchor" href="#show-partitions-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Show Partitions</h3>
+<p>The following command is executed to get the partition information of the table</p>
+<pre><code>SHOW PARTITIONS [db_name.]table_name
+</code></pre>
+<h3>
+<a id="add-a-new-partition" class="anchor" href="#add-a-new-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Add a new partition</h3>
+<pre><code>ALTER TABLE [db_name].table_name ADD PARTITION('new_partition')
+</code></pre>
+<h3>
+<a id="split-a-partition" class="anchor" href="#split-a-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Split a partition</h3>
+<pre><code>ALTER TABLE [db_name].table_name SPLIT PARTITION(partition_id) INTO('new_partition1', 'new_partition2'...)
+</code></pre>
+<h3>
+<a id="drop-a-partition" class="anchor" href="#drop-a-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Drop a partition</h3>
+<p>Only drop partition definition, but keep data</p>
+<pre><code>  ALTER TABLE [db_name].table_name DROP PARTITION(partition_id)
+</code></pre>
+<p>Drop both partition definition and data</p>
+<pre><code>ALTER TABLE [db_name].table_name DROP PARTITION(partition_id) WITH DATA
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>Hash partition table is not supported for ADD, SPLIT and DROP commands.</li>
+<li>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.</li>
+</ul>
+<pre><code>SegmentDir/0_batchno0-0-1502703086921.carbonindex
+          ^
+SegmentDir/part-0-0_batchno0-0-1502703086921.carbondata
+                   ^
+</code></pre>
+<p>Here are some useful tips to improve query performance of carbonData partition table:</p>
+<ul>
+<li>The partitioned column can be excluded from SORT_COLUMNS, this will let other columns to do the efficient sorting.</li>
+<li>When writing SQL on a partition table, try to use filters on the partition column.</li>
+</ul>
+<h2>
+<a id="bucketing" class="anchor" href="#bucketing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>BUCKETING</h2>
+<p>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.</p>
+<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
+                  [(col_name data_type, ...)]
+STORED BY 'carbondata'
+TBLPROPERTIES('BUCKETNUMBER'='noOfBuckets',
+'BUCKETCOLUMNS'='columnname')
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>Bucketing cannot be performed for columns of Complex Data Types.</li>
+<li>Columns in the BUCKETCOLUMN parameter must be dimensions. The BUCKETCOLUMN parameter cannot be a measure or a combination of measures and dimensions.</li>
+</ul>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>


[10/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/dml-of-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/dml-of-carbondata.html b/src/main/webapp/dml-of-carbondata.html
new file mode 100644
index 0000000..2048c9e
--- /dev/null
+++ b/src/main/webapp/dml-of-carbondata.html
@@ -0,0 +1,588 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="carbondata-data-manipulation-language" class="anchor" href="#carbondata-data-manipulation-language" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Data Manipulation Language</h1>
+<p>CarbonData DML statements are documented here,which includes:</p>
+<ul>
+<li><a href="#load-data">LOAD DATA</a></li>
+<li><a href="#insert-data-into-carbondata-table">INSERT DATA</a></li>
+<li><a href="#load-data-using-static-partition">Load Data Using Static Partition</a></li>
+<li><a href="#load-data-using-dynamic-partition">Load Data Using Dynamic Partition</a></li>
+<li><a href="#update-and-delete">UPDATE AND DELETE</a></li>
+<li><a href="#compaction">COMPACTION</a></li>
+<li><a href="./segment-management-on-carbondata.html">SEGMENT MANAGEMENT</a></li>
+</ul>
+<h2>
+<a id="load-data" class="anchor" href="#load-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LOAD DATA</h2>
+<h3>
+<a id="load-files-to-carbondata-table" class="anchor" href="#load-files-to-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LOAD FILES TO CARBONDATA TABLE</h3>
+<p>This command is used to load csv files to carbondata, OPTIONS are not mandatory for data loading process.</p>
+<pre><code>LOAD DATA [LOCAL] INPATH 'folder_path' 
+INTO TABLE [db_name.]table_name 
+OPTIONS(property_name=property_value, ...)
+</code></pre>
+<p><strong>Supported Properties:</strong> <a href="#delimiter">DELIMITER</a>, <a href="#quotechar">QUOTECHAR</a>, <a href="#commentchar">COMMENTCHAR</a>, <a href="#header">HEADER</a>, <a href="#fileheader">FILEHEADER</a>, <a href="#multiline">MULTILINE</a>, <a href="#escapechar">ESCAPECHAR</a>, <a href="#skip_empty_line">SKIP_EMPTY_LINE</a>, <a href="#complex_delimiter_level_1">COMPLEX_DELIMITER_LEVEL_1</a>, <a href="#complex_delimiter_level_2">COMPLEX_DELIMITER_LEVEL_2</a>, <a href="#all_dictionary_path">ALL_DICTIONARY_PATH</a>, <a href="#columndict">COLUMNDICT</a>, <a href="#dateformat">DATEFORMAT</a>,<a href="#timestampformat"> TIMESTAMPFORMAT</a>, <a href="#sort-column-bounds">SORT_COLUMN_BOUNDS</a>, <a href="#single_pass">SINGLE_PASS</a>, <a href="#bad-records-handling">BAD_RECORDS_LOGGER_ENABLE</a>, <a href="#bad-records-handling">BAD_RECORD_PATH</a>, <a href="#bad-records-handling">BAD_RECORDS_ACTION</a>, <a href="#bad-records-handling">IS_EMPTY_DATA_BAD_RECORD</a>, <a href="
 #global_sort_partitions">GLOBAL_SORT_PARTITIONS</a></p>
+<p>You can use the following options to load data:</p>
+<ul>
+<li>
+<h5>
+<a id="delimiter" class="anchor" href="#delimiter" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELIMITER:</h5>
+<p>Delimiters can be provided in the load command.</p>
+<pre><code>OPTIONS('DELIMITER'=',')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="quotechar" class="anchor" href="#quotechar" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>QUOTECHAR:</h5>
+<p>Quote Characters can be provided in the load command.</p>
+<pre><code>OPTIONS('QUOTECHAR'='"')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="commentchar" class="anchor" href="#commentchar" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COMMENTCHAR:</h5>
+<p>Comment Characters can be provided in the load command if user want to comment lines.</p>
+<pre><code>OPTIONS('COMMENTCHAR'='#')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="header" class="anchor" href="#header" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>HEADER:</h5>
+<p>When you load the CSV file without the file header and the file header is the same with the table schema, then add 'HEADER'='false' to load data SQL as user need not provide the file header. By default the value is 'true'.
+false: CSV file is without file header.
+true: CSV file is with file header.</p>
+<pre><code>OPTIONS('HEADER'='false') 
+</code></pre>
+<p><strong>NOTE:</strong> If the HEADER option exist and is set to 'true', then the FILEHEADER option is not required.</p>
+</li>
+<li>
+<h5>
+<a id="fileheader" class="anchor" href="#fileheader" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>FILEHEADER:</h5>
+<p>Headers can be provided in the LOAD DATA command if headers are missing in the source files.</p>
+<pre><code>OPTIONS('FILEHEADER'='column1,column2') 
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="multiline" class="anchor" href="#multiline" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>MULTILINE:</h5>
+<p>CSV with new line character in quotes.</p>
+<pre><code>OPTIONS('MULTILINE'='true') 
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="escapechar" class="anchor" href="#escapechar" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>ESCAPECHAR:</h5>
+<p>Escape char can be provided if user want strict validation of escape character in CSV files.</p>
+<pre><code>OPTIONS('ESCAPECHAR'='\') 
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="skip_empty_line" class="anchor" href="#skip_empty_line" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SKIP_EMPTY_LINE:</h5>
+<p>This option will ignore the empty line in the CSV file during the data load.</p>
+<pre><code>OPTIONS('SKIP_EMPTY_LINE'='TRUE/FALSE') 
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="complex_delimiter_level_1" class="anchor" href="#complex_delimiter_level_1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COMPLEX_DELIMITER_LEVEL_1:</h5>
+<p>Split the complex type data column in a row (eg., a$b$c --&gt; Array = {a,b,c}).</p>
+<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_1'='$') 
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="complex_delimiter_level_2" class="anchor" href="#complex_delimiter_level_2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COMPLEX_DELIMITER_LEVEL_2:</h5>
+<p>Split the complex type nested data column in a row. Applies level_1 delimiter &amp; applies level_2 based on complex data type (eg., a:b$c:d --&gt; Array&gt; = {{a,b},{c,d}}).</p>
+<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_2'=':')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="all_dictionary_path" class="anchor" href="#all_dictionary_path" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>ALL_DICTIONARY_PATH:</h5>
+<p>All dictionary files path.</p>
+<pre><code>OPTIONS('ALL_DICTIONARY_PATH'='/opt/alldictionary/data.dictionary')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="columndict" class="anchor" href="#columndict" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COLUMNDICT:</h5>
+<p>Dictionary file path for specified column.</p>
+<pre><code>OPTIONS('COLUMNDICT'='column1:dictionaryFilePath1,column2:dictionaryFilePath2')
+</code></pre>
+<p><strong>NOTE:</strong> ALL_DICTIONARY_PATH and COLUMNDICT can't be used together.</p>
+</li>
+<li>
+<h5>
+<a id="dateformattimestampformat" class="anchor" href="#dateformattimestampformat" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DATEFORMAT/TIMESTAMPFORMAT:</h5>
+<p>Date and Timestamp format for specified column.</p>
+<pre><code>OPTIONS('DATEFORMAT' = 'yyyy-MM-dd','TIMESTAMPFORMAT'='yyyy-MM-dd HH:mm:ss')
+</code></pre>
+<p><strong>NOTE:</strong> Date formats are specified by date pattern strings. The date pattern letters in CarbonData are same as in JAVA. Refer to <a href="http://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html" target=_blank rel="nofollow">SimpleDateFormat</a>.</p>
+</li>
+<li>
+<h5>
+<a id="sort-column-bounds" class="anchor" href="#sort-column-bounds" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SORT COLUMN BOUNDS:</h5>
+<p>Range bounds for sort columns.</p>
+<p>Suppose the table is created with 'SORT_COLUMNS'='name,id' and the range for name is aaa<del>zzz, the value range for id is 0</del>1000. Then during data loading, we can specify the following option to enhance data loading performance.</p>
+<pre><code>OPTIONS('SORT_COLUMN_BOUNDS'='f,250;l,500;r,750')
+</code></pre>
+<p>Each bound is separated by ';' and each field value in bound is separated by ','. In the example above, we provide 3 bounds to distribute records to 4 partitions. The values 'f','l','r' can evenly distribute the records. Inside carbondata, for a record we compare the value of sort columns with that of the bounds and decide which partition the record will be forwarded to.</p>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>SORT_COLUMN_BOUNDS will be used only when the SORT_SCOPE is 'local_sort'.</li>
+<li>Carbondata will use these bounds as ranges to process data concurrently during the final sort percedure. The records will be sorted and written out inside each partition. Since the partition is sorted, all records will be sorted.</li>
+<li>Since the actual order and literal order of the dictionary column are not necessarily the same, we do not recommend you to use this feature if the first sort column is 'dictionary_include'.</li>
+<li>The option works better if your CPU usage during loading is low. If your system is already CPU tense, better not to use this option. Besides, it depends on the user to specify the bounds. If user does not know the exactly bounds to make the data distributed evenly among the bounds, loading performance will still be better than before or at least the same as before.</li>
+<li>Users can find more information about this option in the description of PR1953.</li>
+</ul>
+</li>
+<li>
+<h5>
+<a id="single_pass" class="anchor" href="#single_pass" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SINGLE_PASS:</h5>
+<p>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.</p>
+</li>
+</ul>
+<p>This option specifies whether to use single pass for loading data or not. By default this option is set to FALSE.</p>
+<pre><code> OPTIONS('SINGLE_PASS'='TRUE')
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>If this option is set to TRUE then data loading will take less time.</li>
+<li>If this option is set to some invalid value other than TRUE or FALSE then it uses the default value.</li>
+</ul>
+<p>Example:</p>
+<pre><code>LOAD DATA local inpath '/opt/rawdata/data.csv' INTO table carbontable
+options('DELIMITER'=',', 'QUOTECHAR'='"','COMMENTCHAR'='#',
+'HEADER'='false',
+'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')
+</code></pre>
+<ul>
+<li>
+<h5>
+<a id="bad-records-handling" class="anchor" href="#bad-records-handling" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>BAD RECORDS HANDLING:</h5>
+<p>Methods of handling bad records are as follows:</p>
+<ul>
+<li>Load all of the data before dealing with the errors.</li>
+<li>Clean or delete bad records before loading data or stop the loading when bad records are found.</li>
+</ul>
+<pre><code>OPTIONS('BAD_RECORDS_LOGGER_ENABLE'='true', 'BAD_RECORD_PATH'='hdfs://hacluster/tmp/carbon', 'BAD_RECORDS_ACTION'='REDIRECT', 'IS_EMPTY_DATA_BAD_RECORD'='false')
+</code></pre>
+</li>
+</ul>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>BAD_RECORDS_ACTION property can have four type of actions for bad records FORCE, REDIRECT, IGNORE and FAIL.</li>
+<li>FAIL option is its Default value. If the FAIL option is used, then data loading fails if any bad records are found.</li>
+<li>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.</li>
+<li>If the FORCE option is used, then it auto-converts the data by storing the bad records as NULL before Loading data.</li>
+<li>If the IGNORE option is used, then bad records are neither loaded nor written to the separate CSV file.</li>
+<li>In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails.</li>
+<li>The default maximum number of characters per column is 32000. If there are more than 32000 characters in a column, please refer to <em>String longer than 32000 characters</em> section.</li>
+<li>Since Bad Records Path can be specified in create, load and carbon properties.
+Therefore, value specified in load will have the highest priority, and value specified in carbon properties will have the least priority.</li>
+</ul>
+<p><strong>Bad Records Path:</strong>
+This property is used to specify the location where bad records would be written.</p>
+<pre><code>TBLPROPERTIES('BAD_RECORDS_PATH'='/opt/badrecords'')
+</code></pre>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<ul>
+<li>
+<h5>
+<a id="global_sort_partitions" class="anchor" href="#global_sort_partitions" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>GLOBAL_SORT_PARTITIONS:</h5>
+<p>If the SORT_SCOPE is defined as GLOBAL_SORT, then user can specify the number of partitions to use while shuffling data for sort using GLOBAL_SORT_PARTITIONS. If it is not configured, or configured less than 1, then it uses the number of map task as reduce task. It is recommended that each reduce task deal with 512MB-1GB data.</p>
+</li>
+</ul>
+<pre><code>OPTIONS('GLOBAL_SORT_PARTITIONS'='2')
+</code></pre>
+<p>NOTE:</p>
+<ul>
+<li>GLOBAL_SORT_PARTITIONS should be Integer type, the range is [1,Integer.MaxValue].</li>
+<li>It is only used when the SORT_SCOPE is GLOBAL_SORT.</li>
+</ul>
+<h3>
+<a id="insert-data-into-carbondata-table" class="anchor" href="#insert-data-into-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>INSERT DATA INTO CARBONDATA TABLE</h3>
+<p>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.</p>
+<pre><code>INSERT INTO TABLE &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p>You can also omit the <code>table</code> keyword and write your query as:</p>
+<pre><code>INSERT INTO &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p>Overwrite insert data:</p>
+<pre><code>INSERT OVERWRITE TABLE &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>The source table and the CarbonData table must have the same table schema.</li>
+<li>The data type of source and destination table columns should be same</li>
+<li>INSERT INTO command does not support partial success if bad records are found, it will fail.</li>
+<li>Data cannot be loaded or updated in source table while insert from source table to target table is in progress.</li>
+</ul>
+<p>Examples</p>
+<pre><code>INSERT INTO table1 SELECT item1, sum(item2 + 1000) as result FROM table2 group by item1
+</code></pre>
+<pre><code>INSERT INTO table1 SELECT item1, item2, item3 FROM table2 where item2='xyz'
+</code></pre>
+<pre><code>INSERT OVERWRITE TABLE table1 SELECT * FROM TABLE2
+</code></pre>
+<h3>
+<a id="load-data-using-static-partition" class="anchor" href="#load-data-using-static-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Load Data Using Static Partition</h3>
+<p>This command allows you to load data using static partition.</p>
+<pre><code>LOAD DATA [LOCAL] INPATH 'folder_path' 
+INTO TABLE [db_name.]table_name PARTITION (partition_spec) 
+OPTIONS(property_name=property_value, ...)    
+INSERT INTO INTO TABLE [db_name.]table_name PARTITION (partition_spec) &lt;SELECT STATEMENT&gt;
+</code></pre>
+<p>Example:</p>
+<pre><code>LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.csv'
+INTO TABLE locationTable
+PARTITION (country = 'US', state = 'CA')  
+INSERT INTO TABLE locationTable
+PARTITION (country = 'US', state = 'AL')
+SELECT &lt;columns list excluding partition columns&gt; FROM another_user
+</code></pre>
+<h3>
+<a id="load-data-using-dynamic-partition" class="anchor" href="#load-data-using-dynamic-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Load Data Using Dynamic Partition</h3>
+<p>This command allows you to load data using dynamic partition. If partition spec is not specified, then the partition is considered as dynamic.</p>
+<p>Example:</p>
+<pre><code>LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.csv'
+INTO TABLE locationTable          
+INSERT INTO TABLE locationTable
+SELECT &lt;columns list excluding partition columns&gt; FROM another_user
+</code></pre>
+<h2>
+<a id="update-and-delete" class="anchor" href="#update-and-delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>UPDATE AND DELETE</h2>
+<h3>
+<a id="update" class="anchor" href="#update" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>UPDATE</h3>
+<p>This command will allow to update the CarbonData table based on the column expression and optional filter conditions.</p>
+<pre><code>UPDATE &lt;table_name&gt; 
+SET (column_name1, column_name2, ... column_name n) = (column1_expression , column2_expression, ... column n_expression )
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p>alternatively the following command can also be used for updating the CarbonData Table :</p>
+<pre><code>UPDATE &lt;table_name&gt;
+SET (column_name1, column_name2) =(select sourceColumn1, sourceColumn2 from sourceTable [ WHERE { &lt;filter_condition&gt; } ] )
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p><strong>NOTE:</strong> The update command fails if multiple input rows in source table are matched with single row in destination table.</p>
+<p>Examples:</p>
+<pre><code>UPDATE t3 SET (t3_salary) = (t3_salary + 9) WHERE t3_name = 'aaa1'
+</code></pre>
+<pre><code>UPDATE t3 SET (t3_date, t3_country) = ('2017-11-18', 'india') WHERE t3_salary &lt; 15003
+</code></pre>
+<pre><code>UPDATE t3 SET (t3_country, t3_name) = (SELECT t5_country, t5_name FROM t5 WHERE t5_id = 5) WHERE t3_id &lt; 5
+</code></pre>
+<pre><code>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 &lt; 5
+</code></pre>
+<pre><code>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 &gt;6
+</code></pre>
+<p>NOTE: Update Complex datatype columns is not supported.</p>
+<h3>
+<a id="delete" class="anchor" href="#delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE</h3>
+<p>This command allows us to delete records from CarbonData table.</p>
+<pre><code>DELETE FROM table_name [WHERE expression]
+</code></pre>
+<p>Examples:</p>
+<pre><code>DELETE FROM carbontable WHERE column1  = 'china'
+</code></pre>
+<pre><code>DELETE FROM carbontable WHERE column1 IN ('china', 'USA')
+</code></pre>
+<pre><code>DELETE FROM carbontable WHERE column1 IN (SELECT column11 FROM sourceTable2)
+</code></pre>
+<pre><code>DELETE FROM carbontable WHERE column1 IN (SELECT column11 FROM sourceTable2 WHERE column1 = 'USA')
+</code></pre>
+<h2>
+<a id="compaction" class="anchor" href="#compaction" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COMPACTION</h2>
+<p>Compaction improves the query performance significantly.</p>
+<p>There are several types of compaction.</p>
+<pre><code>ALTER TABLE [db_name.]table_name COMPACT 'MINOR/MAJOR/CUSTOM'
+</code></pre>
+<ul>
+<li><strong>Minor Compaction</strong></li>
+</ul>
+<p>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:</p>
+<ul>
+<li>Level 1: Merging of the segments which are not yet compacted.</li>
+<li>Level 2: Merging of the compacted segments again to form a larger segment.</li>
+</ul>
+<pre><code>ALTER TABLE table_name COMPACT 'MINOR'
+</code></pre>
+<ul>
+<li><strong>Major Compaction</strong></li>
+</ul>
+<p>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.
+Configure the property carbon.major.compaction.size with appropriate value in MB.</p>
+<p>This command merges the specified number of segments into one segment:</p>
+<pre><code>ALTER TABLE table_name COMPACT 'MAJOR'
+</code></pre>
+<ul>
+<li><strong>Custom Compaction</strong></li>
+</ul>
+<p>In Custom compaction, user can directly specify segment ids to be merged into one large segment.
+All specified segment ids should exist and be valid, otherwise compaction will fail.
+Custom compaction is usually done during the off-peak time.</p>
+<pre><code>ALTER TABLE table_name COMPACT 'CUSTOM' WHERE SEGMENT.ID IN (2,3,4)
+</code></pre>
+<p>NOTE: Compaction is unsupported for table containing Complex columns.</p>
+<ul>
+<li><strong>CLEAN SEGMENTS AFTER Compaction</strong></li>
+</ul>
+<p>Clean the segments which are compacted:</p>
+<pre><code>CLEAN FILES FOR TABLE carbon_table
+</code></pre>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/documentation.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/documentation.html b/src/main/webapp/documentation.html
new file mode 100644
index 0000000..634da94
--- /dev/null
+++ b/src/main/webapp/documentation.html
@@ -0,0 +1,256 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="apache-carbondata-documentation" class="anchor" href="#apache-carbondata-documentation" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Apache CarbonData Documentation</h1>
+<p>Apache CarbonData is a new big data file format for faster interactive query using advanced columnar storage, index, compression and encoding techniques to improve computing efficiency, which helps in speeding up queries by an order of magnitude faster over PetaBytes of data.</p>
+<h2>
+<a id="getting-started" class="anchor" href="#getting-started" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Getting Started</h2>
+<p><strong>File Format Concepts:</strong> Start with the basics of understanding the <a href="./file-structure-of-carbondata.html#carbondata-file-structure">CarbonData file format</a> and its storage structure.This will help to understand other parts of the documentation, incuding deployment, programming and usage guides.</p>
+<p><strong>Quick Start:</strong> <a href="./quick-start-guide.html#installing-and-configuring-carbondata-to-run-locally-with-spark-shell">Run an example program</a> on your local machine or <a href="https://github.com/apache/carbondata/tree/master/examples/spark2/src/main/scala/org/apache/carbondata/examples" target=_blank>study some examples</a>.</p>
+<p><strong>CarbonData SQL Language Reference:</strong> CarbonData extends the Spark SQL language and adds several <a href="./ddl-of-carbondata.html">DDL</a> and <a href="./dml-of-carbondata.html">DML</a> statements to support operations on it.Refer to the <a href="./language-manual.html">Reference Manual</a> to understand the supported features and functions.</p>
+<p><strong>Programming Guides:</strong> You can read our guides about <a href="./sdk-guide.html">APIs supported</a> to learn how to integrate CarbonData with your applications.</p>
+<h2>
+<a id="deployment" class="anchor" href="#deployment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deployment</h2>
+<p>CarbonData can be integrated with popular Execution engines like <a href="./quick-start-guide.html#spark">Spark</a> and <a href="./quick-start-guide.html#presto">Presto</a>.Refer to the <a href="./quick-start-guide.html##deployment-modes">Installation and Configuration</a> section to understand all modes of Integrating CarbonData.</p>
+<h2>
+<a id="contributing-to-carbondata" class="anchor" href="#contributing-to-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contributing to CarbonData</h2>
+<p>The Apache CarbonData community welcomes all kinds of contributions from anyone with a passion for
+faster data format.Contributing to CarbonData doesn?t just mean writing code. Helping new users on the mailing list, testing releases, and improving documentation are also welcome.Please follow the <a href="./how-to-contribute-to-apache-carbondata.html">Contributing to CarbonData guidelines</a> before proposing a design or code change.</p>
+<p><strong>Compiling CarbonData:</strong> This <a href="https://github.com/apache/carbondata/tree/master/build" target=_blank>guide</a> will help you to compile and generate the jars for test.</p>
+<h2>
+<a id="external-resources" class="anchor" href="#external-resources" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>External Resources</h2>
+<p><strong>Wiki:</strong> You can read the <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/CarbonData+Home" rel="nofollow">Apache CarbonData wiki</a> page for upcoming release plan, blogs and training materials.</p>
+<p><strong>Summit:</strong> Presentations from past summits and conferences can be found <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609" rel="nofollow">here</a>.</p>
+<p><strong>Blogs:</strong> Blogs by external users can be found <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=67635497" rel="nofollow">here</a>.</p>
+<p><strong>Performance reports:</strong> TPC-H performance reports can be found <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Performance+-+TPCH+Report+of+CarbonData+%281.2+version%29+and+Parquet+on+Spark+Execution+Engine" rel="nofollow">here</a>.</p>
+<p><strong>Trainings:</strong> Training records on design and code flows can be found <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/CarbonData+Training+Materials" rel="nofollow">here</a>.</p>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__intro').addClass('selected'); });
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/errorpage.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/errorpage.html b/src/main/webapp/errorpage.html
index 090dce5..35cc01a 100644
--- a/src/main/webapp/errorpage.html
+++ b/src/main/webapp/errorpage.html
@@ -89,4 +89,4 @@
 
 
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/faq.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/faq.html b/src/main/webapp/faq.html
index cf84d34..52112b7 100644
--- a/src/main/webapp/faq.html
+++ b/src/main/webapp/faq.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -174,6 +225,22 @@
 <li><a href="#Why-all-executors-are-showing-success-in-Spark-UI-even-after-Dataload-command-failed-at-driver-side">Why all executors are showing success in Spark UI even after Dataload command failed at Driver side?</a></li>
 <li><a href="#Why-different-time-zone-result-for-select-query-output-when-query-SDK-writer-output">Why different time zone result for select query output when query SDK writer output?</a></li>
 </ul>
+<h1>
+<a id="troubleshooting" class="anchor" href="#troubleshooting" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>TroubleShooting</h1>
+<ul>
+<li><a href="#Getting-tablestatus.lock-issues-When-loading-data">Getting tablestatus.lock issues When loading data</a></li>
+<li><a href="#failed-to-load-thrift-libraries">Failed to load thrift libraries</a></li>
+<li><a href="#failed-to-launch-the-spark-shell">Failed to launch the Spark Shell</a></li>
+<li><a href="#failed-to-execute-load-query-on-cluster">Failed to execute load query on cluster</a></li>
+<li><a href="#failed-to-execute-insert-query-on-cluster">Failed to execute insert query on cluster</a></li>
+<li><a href="#failed-to-connect-to-hiveuser-with-thrift">Failed to connect to hiveuser with thrift</a></li>
+<li><a href="#failed-to-read-the-metastore-db-during-table">Failed to read the metastore db during table</a></li>
+<li><a href="#failed-to-load-data-on-the-cluster">Failed to load data on the cluster</a></li>
+<li><a href="#failed-to-insert-data-on-the-cluster">Failed to insert data on the cluster</a></li>
+<li><a href="#failed-to-execute-concurrent-operations-on-table-by-multiple-workers">Failed to execute Concurrent Operations(Load,Insert,Update) on table by multiple workers</a></li>
+<li><a href="#failed-to-create-a-table-with-a-single-numeric-column">Failed to create a table with a single numeric column</a></li>
+</ul>
+<h2></h2>
 <h2>
 <a id="what-are-bad-records" class="anchor" href="#what-are-bad-records" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>What are Bad Records?</h2>
 <p>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.</p>
@@ -311,6 +378,195 @@ If wanted to control timezone of data while writing, then set cluster's time-zon
 <pre><code>cluster timezone is Asia/Shanghai
 TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
 </code></pre>
+<h2>
+<a id="getting-tablestatuslock-issues-when-loading-data" class="anchor" href="#getting-tablestatuslock-issues-when-loading-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Getting tablestatus.lock issues When loading data</h2>
+<p><strong>Symptom</strong></p>
+<pre><code>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.&lt;init&gt;(FileOutputStream.java:213)
+	at java.io.FileOutputStream.&lt;init&gt;(FileOutputStream.java:101)
+</code></pre>
+<p><strong>Possible Cause</strong>
+If you use <code>&lt;hdfs path&gt;</code> as store path when creating carbonsession, may get the errors,because the default is LOCALLOCK.</p>
+<p><strong>Procedure</strong>
+Before creating carbonsession, sets as below:</p>
+<pre><code>import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+CarbonProperties.getInstance().addProperty(CarbonCommonConstants.LOCK_TYPE, "HDFSLOCK")
+</code></pre>
+<h2>
+<a id="failed-to-load-thrift-libraries" class="anchor" href="#failed-to-load-thrift-libraries" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to load thrift libraries</h2>
+<p><strong>Symptom</strong></p>
+<p>Thrift throws following exception :</p>
+<pre><code>thrift: error while loading shared libraries:
+libthriftc.so.0: cannot open shared object file: No such file or directory
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The complete path to the directory containing the libraries is not configured correctly.</p>
+<p><strong>Procedure</strong></p>
+<p>Follow the Apache thrift docs at <a href="https://thrift.apache.org/docs/install" target=_blank rel="nofollow">https://thrift.apache.org/docs/install</a> to install thrift correctly.</p>
+<h2>
+<a id="failed-to-launch-the-spark-shell" class="anchor" href="#failed-to-launch-the-spark-shell" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to launch the Spark Shell</h2>
+<p><strong>Symptom</strong></p>
+<p>The shell prompts the following error :</p>
+<pre><code>org.apache.spark.sql.CarbonContext$$anon$$apache$spark$sql$catalyst$analysis
+$OverrideCatalog$_setter_$org$apache$spark$sql$catalyst$analysis
+$OverrideCatalog$$overrides_$e
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The Spark Version and the selected Spark Profile do not match.</p>
+<p><strong>Procedure</strong></p>
+<ol>
+<li>
+<p>Ensure your spark version and selected profile for spark are correct.</p>
+</li>
+<li>
+<p>Use the following command :</p>
+</li>
+</ol>
+<pre><code>"mvn -Pspark-2.1 -Dspark.version {yourSparkVersion} clean package"
+</code></pre>
+<p>Note :  Refrain from using "mvn clean package" without specifying the profile.</p>
+<h2>
+<a id="failed-to-execute-load-query-on-cluster" class="anchor" href="#failed-to-execute-load-query-on-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute load query on cluster</h2>
+<p><strong>Symptom</strong></p>
+<p>Load query failed with the following exception:</p>
+<pre><code>Dictionary file is locked for updation.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The carbon.properties file is not identical in all the nodes of the cluster.</p>
+<p><strong>Procedure</strong></p>
+<p>Follow the steps to ensure the carbon.properties file is consistent across all the nodes:</p>
+<ol>
+<li>
+<p>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.</p>
+</li>
+<li>
+<p>For the changes to take effect, restart the Spark cluster.</p>
+</li>
+</ol>
+<h2>
+<a id="failed-to-execute-insert-query-on-cluster" class="anchor" href="#failed-to-execute-insert-query-on-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute insert query on cluster</h2>
+<p><strong>Symptom</strong></p>
+<p>Load query failed with the following exception:</p>
+<pre><code>Dictionary file is locked for updation.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The carbon.properties file is not identical in all the nodes of the cluster.</p>
+<p><strong>Procedure</strong></p>
+<p>Follow the steps to ensure the carbon.properties file is consistent across all the nodes:</p>
+<ol>
+<li>
+<p>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.</p>
+</li>
+<li>
+<p>For the changes to take effect, restart the Spark cluster.</p>
+</li>
+</ol>
+<h2>
+<a id="failed-to-connect-to-hiveuser-with-thrift" class="anchor" href="#failed-to-connect-to-hiveuser-with-thrift" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to connect to hiveuser with thrift</h2>
+<p><strong>Symptom</strong></p>
+<p>We get the following exception :</p>
+<pre><code>Cannot connect to hiveuser.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The external process does not have permission to access.</p>
+<p><strong>Procedure</strong></p>
+<p>Ensure that the Hiveuser in mysql must allow its access to the external processes.</p>
+<h2>
+<a id="failed-to-read-the-metastore-db-during-table-creation" class="anchor" href="#failed-to-read-the-metastore-db-during-table-creation" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to read the metastore db during table creation</h2>
+<p><strong>Symptom</strong></p>
+<p>We get the following exception on trying to connect :</p>
+<pre><code>Cannot read the metastore db
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The metastore db is dysfunctional.</p>
+<p><strong>Procedure</strong></p>
+<p>Remove the metastore db from the carbon.metastore in the Spark Directory.</p>
+<h2>
+<a id="failed-to-load-data-on-the-cluster" class="anchor" href="#failed-to-load-data-on-the-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to load data on the cluster</h2>
+<p><strong>Symptom</strong></p>
+<p>Data loading fails with the following exception :</p>
+<pre><code>Data Load failure exception
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The following issue can cause the failure :</p>
+<ol>
+<li>
+<p>The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.</p>
+</li>
+<li>
+<p>Path to hdfs ddl is not configured correctly in the carbon.properties.</p>
+</li>
+</ol>
+<p><strong>Procedure</strong></p>
+<p>Follow the steps to ensure the following configuration files are consistent across all the nodes:</p>
+<ol>
+<li>
+<p>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.</p>
+<p>Note : Set the path to hdfs ddl in carbon.properties in the master node.</p>
+</li>
+<li>
+<p>For the changes to take effect, restart the Spark cluster.</p>
+</li>
+</ol>
+<h2>
+<a id="failed-to-insert-data-on-the-cluster" class="anchor" href="#failed-to-insert-data-on-the-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to insert data on the cluster</h2>
+<p><strong>Symptom</strong></p>
+<p>Insertion fails with the following exception :</p>
+<pre><code>Data Load failure exception
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The following issue can cause the failure :</p>
+<ol>
+<li>
+<p>The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.</p>
+</li>
+<li>
+<p>Path to hdfs ddl is not configured correctly in the carbon.properties.</p>
+</li>
+</ol>
+<p><strong>Procedure</strong></p>
+<p>Follow the steps to ensure the following configuration files are consistent across all the nodes:</p>
+<ol>
+<li>
+<p>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.</p>
+<p>Note : Set the path to hdfs ddl in carbon.properties in the master node.</p>
+</li>
+<li>
+<p>For the changes to take effect, restart the Spark cluster.</p>
+</li>
+</ol>
+<h2>
+<a id="failed-to-execute-concurrent-operations-on-table-by-multiple-workers" class="anchor" href="#failed-to-execute-concurrent-operations-on-table-by-multiple-workers" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute Concurrent Operations on table by multiple workers</h2>
+<p><strong>Symptom</strong></p>
+<p>Execution fails with the following exception :</p>
+<pre><code>Table is locked for updation.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>Concurrency not supported.</p>
+<p><strong>Procedure</strong></p>
+<p>Worker must wait for the query execution to complete and the table to release the lock for another query execution to succeed.</p>
+<h2>
+<a id="failed-to-create-a-table-with-a-single-numeric-column" class="anchor" href="#failed-to-create-a-table-with-a-single-numeric-column" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to create a table with a single numeric column</h2>
+<p><strong>Symptom</strong></p>
+<p>Execution fails with the following exception :</p>
+<pre><code>Table creation fails.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>Behaviour not supported.</p>
+<p><strong>Procedure</strong></p>
+<p>A single column that can be considered as dimension is mandatory for table creation.</p>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__faq').addClass('selected'); });
+</script>
 </div>
 </div>
 </div>
@@ -326,4 +582,4 @@ TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/file-structure-of-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/file-structure-of-carbondata.html b/src/main/webapp/file-structure-of-carbondata.html
index edebccf..b96d622 100644
--- a/src/main/webapp/file-structure-of-carbondata.html
+++ b/src/main/webapp/file-structure-of-carbondata.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -168,7 +219,7 @@
 <ul>
 <li>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.</li>
 </ul>
-<p><a href="../docs/images/carbon_data_file_structure_new.png?raw=true" target="_blank"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_file_structure_new.png?raw=true" alt="CarbonData File Structure" style="max-width:100%;"></a></p>
+<p><a href="../docs/images/carbon_data_file_structure_new.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_file_structure_new.png?raw=true" alt="CarbonData File Structure" style="max-width:100%;"></a></p>
 <ul>
 <li>File Header : It contains CarbonData file version number, list of column schema and schema updation timestamp.</li>
 <li>File Footer : it contains Number of rows, segmentinfo ,all blocklets? info and index, you can find the detail from the below diagram.</li>
@@ -176,7 +227,7 @@
 <li>Column Page Group : Data of one column and it is further divided into pages, it is guaranteed to be contiguous in file.</li>
 <li>Page : It has the data of one column and the number of row is fixed to 32000 size.</li>
 </ul>
-<p><a href="../docs/images/carbon_data_format_new.png?raw=true" target="_blank"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_format_new.png?raw=true" alt="CarbonData File Format" style="max-width:100%;"></a></p>
+<p><a href="../docs/images/carbon_data_format_new.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_format_new.png?raw=true" alt="CarbonData File Format" style="max-width:100%;"></a></p>
 <h3>
 <a id="each-page-contains-three-types-of-data" class="anchor" href="#each-page-contains-three-types-of-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Each page contains three types of data</h3>
 <ul>
@@ -184,6 +235,17 @@
 <li>Row ID Page (optional): Contains the row ID mappings used when the data page is stored as an inverted index.</li>
 <li>RLE Page (optional): Contains additional metadata used when the data page is RLE coded.</li>
 </ul>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -199,4 +261,4 @@
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>


[36/39] carbondata-site git commit: refactor document

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/bee56334/content/useful-tips-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/useful-tips-on-carbondata.html b/content/useful-tips-on-carbondata.html
new file mode 100644
index 0000000..912cb48
--- /dev/null
+++ b/content/useful-tips-on-carbondata.html
@@ -0,0 +1,480 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="row">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="useful-tips" class="anchor" href="#useful-tips" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Useful Tips</h1>
+<p>This tutorial guides you to create CarbonData Tables and optimize performance.
+The following sections will elaborate on the below topics :</p>
+<ul>
+<li><a href="#suggestions-to-create-carbondata-table">Suggestions to create CarbonData Table</a></li>
+<li><a href="#configuration-for-optimizing-data-loading-performance-for-massive-data">Configuration for Optimizing Data Loading performance for Massive Data</a></li>
+<li><a href="#configurations-for-optimizing-carbondata-performance">Optimizing Mass Data Loading</a></li>
+</ul>
+<h2>
+<a id="suggestions-to-create-carbondata-table" class="anchor" href="#suggestions-to-create-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Suggestions to Create CarbonData Table</h2>
+<p>For example, the results of the analysis for table creation with dimensions ranging from 10 thousand to 10 billion rows and 100 to 300 columns have been summarized below.
+The following table describes some of the columns from the table used.</p>
+<ul>
+<li><strong>Table Column Description</strong></li>
+</ul>
+<table>
+<thead>
+<tr>
+<th>Column Name</th>
+<th>Data Type</th>
+<th>Cardinality</th>
+<th>Attribution</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>msisdn</td>
+<td>String</td>
+<td>30 million</td>
+<td>Dimension</td>
+</tr>
+<tr>
+<td>BEGIN_TIME</td>
+<td>BigInt</td>
+<td>10 Thousand</td>
+<td>Dimension</td>
+</tr>
+<tr>
+<td>HOST</td>
+<td>String</td>
+<td>1 million</td>
+<td>Dimension</td>
+</tr>
+<tr>
+<td>Dime_1</td>
+<td>String</td>
+<td>1 Thousand</td>
+<td>Dimension</td>
+</tr>
+<tr>
+<td>counter_1</td>
+<td>Decimal</td>
+<td>NA</td>
+<td>Measure</td>
+</tr>
+<tr>
+<td>counter_2</td>
+<td>Numeric(20,0)</td>
+<td>NA</td>
+<td>Measure</td>
+</tr>
+<tr>
+<td>...</td>
+<td>...</td>
+<td>NA</td>
+<td>Measure</td>
+</tr>
+<tr>
+<td>counter_100</td>
+<td>Decimal</td>
+<td>NA</td>
+<td>Measure</td>
+</tr>
+</tbody>
+</table>
+<ul>
+<li><strong>Put the frequently-used column filter in the beginning</strong></li>
+</ul>
+<p>For example, MSISDN filter is used in most of the query then we must put the MSISDN in the first column.
+The create table command can be modified as suggested below :</p>
+<pre><code>create table carbondata_table(
+  msisdn String,
+  BEGIN_TIME bigint,
+  HOST String,
+  Dime_1 String,
+  counter_1, Decimal
+  ...
+  
+  )STORED BY 'carbondata'
+  TBLPROPERTIES ('SORT_COLUMNS'='msisdn, Dime_1')
+</code></pre>
+<p>Now the query with MSISDN in the filter will be more efficient.</p>
+<ul>
+<li><strong>Put the frequently-used columns in the order of low to high cardinality</strong></li>
+</ul>
+<p>If the table in the specified query has multiple columns which are frequently used to filter the results, it is suggested to put
+the columns in the order of cardinality low to high. This ordering of frequently used columns improves the compression ratio and
+enhances the performance of queries with filter on these columns.</p>
+<p>For example, if MSISDN, HOST and Dime_1 are frequently-used columns, then the column order of table is suggested as
+Dime_1&gt;HOST&gt;MSISDN, because Dime_1 has the lowest cardinality.
+The create table command can be modified as suggested below :</p>
+<pre><code>create table carbondata_table(
+    msisdn String,
+    BEGIN_TIME bigint,
+    HOST String,
+    Dime_1 String,
+    counter_1, Decimal
+    ...
+    
+    )STORED BY 'carbondata'
+    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
+</code></pre>
+<ul>
+<li><strong>For measure type columns with non high accuracy, replace Numeric(20,0) data type with Double data type</strong></li>
+</ul>
+<p>For columns of measure type, not requiring high accuracy, it is suggested to replace Numeric data type with Double to enhance query performance.
+The create table command can be modified as below :</p>
+<pre><code>  create table carbondata_table(
+    Dime_1 String,
+    BEGIN_TIME bigint,
+    END_TIME bigint,
+    HOST String,
+    MSISDN String,
+    counter_1 decimal,
+    counter_2 double,
+    ...
+    )STORED BY 'carbondata'
+    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
+</code></pre>
+<p>The result of performance analysis of test-case shows reduction in query execution time from 15 to 3 seconds, thereby improving performance by nearly 5 times.</p>
+<ul>
+<li><strong>Columns of incremental character should be re-arranged at the end of dimensions</strong></li>
+</ul>
+<p>Consider the following scenario where data is loaded each day and the begin_time is incremental for each load, it is suggested to put begin_time at the end of dimensions.
+Incremental values are efficient in using min/max index. The create table command can be modified as below :</p>
+<pre><code>create table carbondata_table(
+  Dime_1 String,
+  HOST String,
+  MSISDN String,
+  counter_1 double,
+  counter_2 double,
+  BEGIN_TIME bigint,
+  END_TIME bigint,
+  ...
+  counter_100 double
+  )STORED BY 'carbondata'
+  TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>BloomFilter can be created to enhance performance for queries with precise equal/in conditions. You can find more information about it in BloomFilter datamap <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/bloomfilter-datamap-guide.html" target=_blank>document</a>.</li>
+</ul>
+<h2>
+<a id="configuration-for-optimizing-data-loading-performance-for-massive-data" class="anchor" href="#configuration-for-optimizing-data-loading-performance-for-massive-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configuration for Optimizing Data Loading performance for Massive Data</h2>
+<p>CarbonData supports large data load, in this process sorting data while loading consumes a lot of memory and disk IO and
+this can result sometimes in "Out Of Memory" exception.
+If you do not have much memory to use, then you may prefer to slow the speed of data loading instead of data load failure.
+You can configure CarbonData by tuning following properties in carbon.properties file to get a better performance.</p>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Default Value</th>
+<th>Description/Tuning</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>carbon.number.of.cores.while.loading</td>
+<td>Default: 2.This value should be &gt;= 2</td>
+<td>Specifies the number of cores used for data processing during data loading in CarbonData.</td>
+</tr>
+<tr>
+<td>carbon.sort.size</td>
+<td>Default: 100000. The value should be &gt;= 100.</td>
+<td>Threshold to write local file in sort step when loading data</td>
+</tr>
+<tr>
+<td>carbon.sort.file.write.buffer.size</td>
+<td>Default:  50000.</td>
+<td>DataOutputStream buffer.</td>
+</tr>
+<tr>
+<td>carbon.number.of.cores.block.sort</td>
+<td>Default: 7</td>
+<td>If you have huge memory and CPUs, increase it as you will</td>
+</tr>
+<tr>
+<td>carbon.merge.sort.reader.thread</td>
+<td>Default: 3</td>
+<td>Specifies the number of cores used for temp file merging during data loading in CarbonData.</td>
+</tr>
+<tr>
+<td>carbon.merge.sort.prefetch</td>
+<td>Default: true</td>
+<td>You may want set this value to false if you have not enough memory</td>
+</tr>
+</tbody>
+</table>
+<p>For example, if there are 10 million records, and i have only 16 cores, 64GB memory, will be loaded to CarbonData table.
+Using the default configuration  always fail in sort step. Modify carbon.properties as suggested below:</p>
+<pre><code>carbon.number.of.cores.block.sort=1
+carbon.merge.sort.reader.thread=1
+carbon.sort.size=5000
+carbon.sort.file.write.buffer.size=5000
+carbon.merge.sort.prefetch=false
+</code></pre>
+<h2>
+<a id="configurations-for-optimizing-carbondata-performance" class="anchor" href="#configurations-for-optimizing-carbondata-performance" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configurations for Optimizing CarbonData Performance</h2>
+<p>Recently we did some performance POC on CarbonData for Finance and telecommunication Field. It involved detailed queries and aggregation
+scenarios. After the completion of POC, some of the configurations impacting the performance have been identified and tabulated below :</p>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Location</th>
+<th>Used For</th>
+<th>Description</th>
+<th>Tuning</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>carbon.sort.intermediate.files.limit</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>During the loading of data, local temp is used to sort the data. This number specifies the minimum number of intermediate files after which the  merge sort has to be initiated.</td>
+<td>Increasing the parameter to a higher value will improve the load performance. For example, when we increase the value from 20 to 100, it increases the data load performance from 35MB/S to more than 50MB/S. Higher values of this parameter consumes  more memory during the load.</td>
+</tr>
+<tr>
+<td>carbon.number.of.cores.while.loading</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Specifies the number of cores used for data processing during data loading in CarbonData.</td>
+<td>If you have more number of CPUs, then you can increase the number of CPUs, which will increase the performance. For example if we increase the value from 2 to 4 then the CSV reading performance can increase about 1 times</td>
+</tr>
+<tr>
+<td>carbon.compaction.level.threshold</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading and Querying</td>
+<td>For minor compaction, specifies the number of segments to be merged in stage 1 and number of compacted segments to be merged in stage 2.</td>
+<td>Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance. Configuring this parameter will merge the small segment to one big segment which will sort the data and improve the performance. For Example in one telecommunication scenario, the performance improves about 2 times after minor compaction.</td>
+</tr>
+<tr>
+<td>spark.sql.shuffle.partitions</td>
+<td>spark/conf/spark-defaults.conf</td>
+<td>Querying</td>
+<td>The number of task started when spark shuffle.</td>
+<td>The value can be 1 to 2 times as much as the executor cores. In an aggregation scenario, reducing the number from 200 to 32 reduced the query time from 17 to 9 seconds.</td>
+</tr>
+<tr>
+<td>spark.executor.instances/spark.executor.cores/spark.executor.memory</td>
+<td>spark/conf/spark-defaults.conf</td>
+<td>Querying</td>
+<td>The number of executors, CPU cores, and memory used for CarbonData query.</td>
+<td>In the bank scenario, we provide the 4 CPUs cores and 15 GB for each executor which can get good performance. This 2 value does not mean more the better. It needs to be configured properly in case of limited resources. For example, In the bank scenario, it has enough CPU 32 cores each node but less memory 64 GB each node. So we cannot give more CPU but less memory. For example, when 4 cores and 12GB for each executor. It sometimes happens GC during the query which impact the query performance very much from the 3 second to more than 15 seconds. In this scenario need to increase the memory or decrease the CPU cores.</td>
+</tr>
+<tr>
+<td>carbon.detail.batch.size</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>The buffer size to store records, returned from the block scan.</td>
+<td>In limit scenario this parameter is very important. For example your query limit is 1000. But if we set this value to 3000 that means we get 3000 records from scan but spark will only take 1000 rows. So the 2000 remaining are useless. In one Finance test case after we set it to 100, in the limit 1000 scenario the performance increase about 2 times in comparison to if we set this value to 12000.</td>
+</tr>
+<tr>
+<td>carbon.use.local.dir</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Whether use YARN local directories for multi-table load disk load balance</td>
+<td>If this is set it to true CarbonData will use YARN local directories for multi-table load disk load balance, that will improve the data load performance.</td>
+</tr>
+<tr>
+<td>carbon.use.multiple.temp.dir</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Whether to use multiple YARN local directories during table data loading for disk load balance</td>
+<td>After enabling 'carbon.use.local.dir', if this is set to true, CarbonData will use all YARN local directories during data load for disk load balance, that will improve the data load performance. Please enable this property when you encounter disk hotspot problem during data loading.</td>
+</tr>
+<tr>
+<td>carbon.sort.temp.compressor</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Specify the name of compressor to compress the intermediate sort temporary files during sort procedure in data loading.</td>
+<td>The optional values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files. This parameter will be useful if you encounter disk bottleneck.</td>
+</tr>
+<tr>
+<td>carbon.load.skewedDataOptimization.enabled</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Whether to enable size based block allocation strategy for data loading.</td>
+<td>When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data -- It's useful if the size of your input data files varies widely, say 1MB~1GB.</td>
+</tr>
+<tr>
+<td>carbon.load.min.size.enabled</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Whether to enable node minumun input data size allocation strategy for data loading.</td>
+<td>When loading, carbondata will use node minumun input data size allocation strategy for task distribution. It will make sure the node load the minimum amount of data -- It's useful if the size of your input data files very small, say 1MB~256MB,Avoid generating a large number of small files.</td>
+</tr>
+</tbody>
+</table>
+<p>Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.</p>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file


[14/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/performance-tuning.html
----------------------------------------------------------------------
diff --git a/content/performance-tuning.html b/content/performance-tuning.html
new file mode 100644
index 0000000..49b3d3a
--- /dev/null
+++ b/content/performance-tuning.html
@@ -0,0 +1,529 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="useful-tips" class="anchor" href="#useful-tips" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Useful Tips</h1>
+<p>This tutorial guides you to create CarbonData Tables and optimize performance.
+The following sections will elaborate on the below topics :</p>
+<ul>
+<li><a href="#suggestions-to-create-carbondata-table">Suggestions to create CarbonData Table</a></li>
+<li><a href="#configuration-for-optimizing-data-loading-performance-for-massive-data">Configuration for Optimizing Data Loading performance for Massive Data</a></li>
+<li><a href="#configurations-for-optimizing-carbondata-performance">Optimizing Query Performance</a></li>
+</ul>
+<h2>
+<a id="suggestions-to-create-carbondata-table" class="anchor" href="#suggestions-to-create-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Suggestions to Create CarbonData Table</h2>
+<p>For example, the results of the analysis for table creation with dimensions ranging from 10 thousand to 10 billion rows and 100 to 300 columns have been summarized below.
+The following table describes some of the columns from the table used.</p>
+<ul>
+<li><strong>Table Column Description</strong></li>
+</ul>
+<table>
+<thead>
+<tr>
+<th>Column Name</th>
+<th>Data Type</th>
+<th>Cardinality</th>
+<th>Attribution</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>msisdn</td>
+<td>String</td>
+<td>30 million</td>
+<td>Dimension</td>
+</tr>
+<tr>
+<td>BEGIN_TIME</td>
+<td>BigInt</td>
+<td>10 Thousand</td>
+<td>Dimension</td>
+</tr>
+<tr>
+<td>HOST</td>
+<td>String</td>
+<td>1 million</td>
+<td>Dimension</td>
+</tr>
+<tr>
+<td>Dime_1</td>
+<td>String</td>
+<td>1 Thousand</td>
+<td>Dimension</td>
+</tr>
+<tr>
+<td>counter_1</td>
+<td>Decimal</td>
+<td>NA</td>
+<td>Measure</td>
+</tr>
+<tr>
+<td>counter_2</td>
+<td>Numeric(20,0)</td>
+<td>NA</td>
+<td>Measure</td>
+</tr>
+<tr>
+<td>...</td>
+<td>...</td>
+<td>NA</td>
+<td>Measure</td>
+</tr>
+<tr>
+<td>counter_100</td>
+<td>Decimal</td>
+<td>NA</td>
+<td>Measure</td>
+</tr>
+</tbody>
+</table>
+<ul>
+<li><strong>Put the frequently-used column filter in the beginning of SORT_COLUMNS</strong></li>
+</ul>
+<p>For example, MSISDN filter is used in most of the query then we must put the MSISDN as the first column in SORT_COLUMNS property.
+The create table command can be modified as suggested below :</p>
+<pre><code>create table carbondata_table(
+  msisdn String,
+  BEGIN_TIME bigint,
+  HOST String,
+  Dime_1 String,
+  counter_1, Decimal
+  ...
+  
+  )STORED BY 'carbondata'
+  TBLPROPERTIES ('SORT_COLUMNS'='msisdn, Dime_1')
+</code></pre>
+<p>Now the query with MSISDN in the filter will be more efficient.</p>
+<ul>
+<li><strong>Put the frequently-used columns in the order of low to high cardinality in SORT_COLUMNS</strong></li>
+</ul>
+<p>If the table in the specified query has multiple columns which are frequently used to filter the results, it is suggested to put
+the columns in the order of cardinality low to high in SORT_COLUMNS configuration. This ordering of frequently used columns improves the compression ratio and
+enhances the performance of queries with filter on these columns.</p>
+<p>For example, if MSISDN, HOST and Dime_1 are frequently-used columns, then the column order of table is suggested as
+Dime_1&gt;HOST&gt;MSISDN, because Dime_1 has the lowest cardinality.
+The create table command can be modified as suggested below :</p>
+<pre><code>create table carbondata_table(
+    msisdn String,
+    BEGIN_TIME bigint,
+    HOST String,
+    Dime_1 String,
+    counter_1, Decimal
+    ...
+    
+    )STORED BY 'carbondata'
+    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
+</code></pre>
+<ul>
+<li><strong>For measure type columns with non high accuracy, replace Numeric(20,0) data type with Double data type</strong></li>
+</ul>
+<p>For columns of measure type, not requiring high accuracy, it is suggested to replace Numeric data type with Double to enhance query performance.
+The create table command can be modified as below :</p>
+<pre><code>  create table carbondata_table(
+    Dime_1 String,
+    BEGIN_TIME bigint,
+    END_TIME bigint,
+    HOST String,
+    MSISDN String,
+    counter_1 decimal,
+    counter_2 double,
+    ...
+    )STORED BY 'carbondata'
+    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
+</code></pre>
+<p>The result of performance analysis of test-case shows reduction in query execution time from 15 to 3 seconds, thereby improving performance by nearly 5 times.</p>
+<ul>
+<li><strong>Columns of incremental character should be re-arranged at the end of dimensions</strong></li>
+</ul>
+<p>Consider the following scenario where data is loaded each day and the begin_time is incremental for each load, it is suggested to put begin_time at the end of dimensions.
+Incremental values are efficient in using min/max index. The create table command can be modified as below :</p>
+<pre><code>create table carbondata_table(
+  Dime_1 String,
+  HOST String,
+  MSISDN String,
+  counter_1 double,
+  counter_2 double,
+  BEGIN_TIME bigint,
+  END_TIME bigint,
+  ...
+  counter_100 double
+  )STORED BY 'carbondata'
+  TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>BloomFilter can be created to enhance performance for queries with precise equal/in conditions. You can find more information about it in BloomFilter datamap <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/bloomfilter-datamap-guide.html" target=_blank>document</a>.</li>
+</ul>
+<h2>
+<a id="configuration-for-optimizing-data-loading-performance-for-massive-data" class="anchor" href="#configuration-for-optimizing-data-loading-performance-for-massive-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configuration for Optimizing Data Loading performance for Massive Data</h2>
+<p>CarbonData supports large data load, in this process sorting data while loading consumes a lot of memory and disk IO and
+this can result sometimes in "Out Of Memory" exception.
+If you do not have much memory to use, then you may prefer to slow the speed of data loading instead of data load failure.
+You can configure CarbonData by tuning following properties in carbon.properties file to get a better performance.</p>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Default Value</th>
+<th>Description/Tuning</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>carbon.number.of.cores.while.loading</td>
+<td>Default: 2.This value should be &gt;= 2</td>
+<td>Specifies the number of cores used for data processing during data loading in CarbonData.</td>
+</tr>
+<tr>
+<td>carbon.sort.size</td>
+<td>Default: 100000. The value should be &gt;= 100.</td>
+<td>Threshold to write local file in sort step when loading data</td>
+</tr>
+<tr>
+<td>carbon.sort.file.write.buffer.size</td>
+<td>Default:  50000.</td>
+<td>DataOutputStream buffer.</td>
+</tr>
+<tr>
+<td>carbon.merge.sort.reader.thread</td>
+<td>Default: 3</td>
+<td>Specifies the number of cores used for temp file merging during data loading in CarbonData.</td>
+</tr>
+<tr>
+<td>carbon.merge.sort.prefetch</td>
+<td>Default: true</td>
+<td>You may want set this value to false if you have not enough memory</td>
+</tr>
+</tbody>
+</table>
+<p>For example, if there are 10 million records, and i have only 16 cores, 64GB memory, will be loaded to CarbonData table.
+Using the default configuration  always fail in sort step. Modify carbon.properties as suggested below:</p>
+<pre><code>carbon.merge.sort.reader.thread=1
+carbon.sort.size=5000
+carbon.sort.file.write.buffer.size=5000
+carbon.merge.sort.prefetch=false
+</code></pre>
+<h2>
+<a id="configurations-for-optimizing-carbondata-performance" class="anchor" href="#configurations-for-optimizing-carbondata-performance" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configurations for Optimizing CarbonData Performance</h2>
+<p>Recently we did some performance POC on CarbonData for Finance and telecommunication Field. It involved detailed queries and aggregation
+scenarios. After the completion of POC, some of the configurations impacting the performance have been identified and tabulated below :</p>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Location</th>
+<th>Used For</th>
+<th>Description</th>
+<th>Tuning</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>carbon.sort.intermediate.files.limit</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>During the loading of data, local temp is used to sort the data. This number specifies the minimum number of intermediate files after which the  merge sort has to be initiated.</td>
+<td>Increasing the parameter to a higher value will improve the load performance. For example, when we increase the value from 20 to 100, it increases the data load performance from 35MB/S to more than 50MB/S. Higher values of this parameter consumes  more memory during the load.</td>
+</tr>
+<tr>
+<td>carbon.number.of.cores.while.loading</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Specifies the number of cores used for data processing during data loading in CarbonData.</td>
+<td>If you have more number of CPUs, then you can increase the number of CPUs, which will increase the performance. For example if we increase the value from 2 to 4 then the CSV reading performance can increase about 1 times</td>
+</tr>
+<tr>
+<td>carbon.compaction.level.threshold</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading and Querying</td>
+<td>For minor compaction, specifies the number of segments to be merged in stage 1 and number of compacted segments to be merged in stage 2.</td>
+<td>Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance. Configuring this parameter will merge the small segment to one big segment which will sort the data and improve the performance. For Example in one telecommunication scenario, the performance improves about 2 times after minor compaction.</td>
+</tr>
+<tr>
+<td>spark.sql.shuffle.partitions</td>
+<td>spark/conf/spark-defaults.conf</td>
+<td>Querying</td>
+<td>The number of task started when spark shuffle.</td>
+<td>The value can be 1 to 2 times as much as the executor cores. In an aggregation scenario, reducing the number from 200 to 32 reduced the query time from 17 to 9 seconds.</td>
+</tr>
+<tr>
+<td>spark.executor.instances/spark.executor.cores/spark.executor.memory</td>
+<td>spark/conf/spark-defaults.conf</td>
+<td>Querying</td>
+<td>The number of executors, CPU cores, and memory used for CarbonData query.</td>
+<td>In the bank scenario, we provide the 4 CPUs cores and 15 GB for each executor which can get good performance. This 2 value does not mean more the better. It needs to be configured properly in case of limited resources. For example, In the bank scenario, it has enough CPU 32 cores each node but less memory 64 GB each node. So we cannot give more CPU but less memory. For example, when 4 cores and 12GB for each executor. It sometimes happens GC during the query which impact the query performance very much from the 3 second to more than 15 seconds. In this scenario need to increase the memory or decrease the CPU cores.</td>
+</tr>
+<tr>
+<td>carbon.detail.batch.size</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>The buffer size to store records, returned from the block scan.</td>
+<td>In limit scenario this parameter is very important. For example your query limit is 1000. But if we set this value to 3000 that means we get 3000 records from scan but spark will only take 1000 rows. So the 2000 remaining are useless. In one Finance test case after we set it to 100, in the limit 1000 scenario the performance increase about 2 times in comparison to if we set this value to 12000.</td>
+</tr>
+<tr>
+<td>carbon.use.local.dir</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Whether use YARN local directories for multi-table load disk load balance</td>
+<td>If this is set it to true CarbonData will use YARN local directories for multi-table load disk load balance, that will improve the data load performance.</td>
+</tr>
+<tr>
+<td>carbon.use.multiple.temp.dir</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Whether to use multiple YARN local directories during table data loading for disk load balance</td>
+<td>After enabling 'carbon.use.local.dir', if this is set to true, CarbonData will use all YARN local directories during data load for disk load balance, that will improve the data load performance. Please enable this property when you encounter disk hotspot problem during data loading.</td>
+</tr>
+<tr>
+<td>carbon.sort.temp.compressor</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Specify the name of compressor to compress the intermediate sort temporary files during sort procedure in data loading.</td>
+<td>The optional values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files. This parameter will be useful if you encounter disk bottleneck.</td>
+</tr>
+<tr>
+<td>carbon.load.skewedDataOptimization.enabled</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Whether to enable size based block allocation strategy for data loading.</td>
+<td>When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data -- It's useful if the size of your input data files varies widely, say 1MB~1GB.</td>
+</tr>
+<tr>
+<td>carbon.load.min.size.enabled</td>
+<td>spark/carbonlib/carbon.properties</td>
+<td>Data loading</td>
+<td>Whether to enable node minumun input data size allocation strategy for data loading.</td>
+<td>When loading, carbondata will use node minumun input data size allocation strategy for task distribution. It will make sure the node load the minimum amount of data -- It's useful if the size of your input data files very small, say 1MB~256MB,Avoid generating a large number of small files.</td>
+</tr>
+</tbody>
+</table>
+<p>Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.</p>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__perf').addClass('selected'); });
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/preaggregate-datamap-guide.html
----------------------------------------------------------------------
diff --git a/content/preaggregate-datamap-guide.html b/content/preaggregate-datamap-guide.html
index d68764d..9220c84 100644
--- a/content/preaggregate-datamap-guide.html
+++ b/content/preaggregate-datamap-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -266,7 +317,7 @@ kinds of DataMap:</p>
 a. 'path' is used to specify the store location of the datamap.('path'='/location/').
 b. 'partitioning' when set to false enables user to disable partitioning of the datamap.
 Default value is true for this property.</li>
-<li>timeseries, for timeseries roll-up table. Please refer to <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/timeseries-datamap-guide.html" target=_blank>Timeseries DataMap</a>
+<li>timeseries, for timeseries roll-up table. Please refer to <a href="./timeseries-datamap-guide.html">Timeseries DataMap</a>
 </li>
 </ol>
 <p>DataMap can be dropped using following DDL</p>
@@ -415,6 +466,17 @@ release, user can do as following:</p>
 <li>Create the pre-aggregate table again by <code>CREATE DATAMAP</code> command
 Basically, user can manually trigger the operation by re-building the datamap.</li>
 </ol>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -430,4 +492,4 @@ Basically, user can manually trigger the operation by re-building the datamap.</
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/quick-start-guide.html
----------------------------------------------------------------------
diff --git a/content/quick-start-guide.html b/content/quick-start-guide.html
index 89380b4..ea88086 100644
--- a/content/quick-start-guide.html
+++ b/content/quick-start-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -161,12 +212,12 @@
                                     <div>
 <h1>
 <a id="quick-start" class="anchor" href="#quick-start" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Quick Start</h1>
-<p>This tutorial provides a quick introduction to using CarbonData.</p>
+<p>This tutorial provides a quick introduction to using CarbonData.To follow along with this guide, first download a packaged release of CarbonData from the <a href="https://dist.apache.org/repos/dist/release/carbondata/" target=_blank rel="nofollow">CarbonData website</a>.Alternatively it can be created following <a href="https://github.com/apache/carbondata/tree/master/build" target=_blank>Building CarbonData</a> steps.</p>
 <h2>
 <a id="prerequisites" class="anchor" href="#prerequisites" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h2>
 <ul>
 <li>
-<p><a href="https://github.com/apache/carbondata/blob/master/build" target=_blank>Installation and building CarbonData</a>.</p>
+<p>Spark 2.2.1 version is installed and running.CarbonData supports Spark versions upto 2.2.1.Please follow steps described in <a href="https://spark.apache.org/docs/latest" target=_blank rel="nofollow">Spark docs website</a> for installing and running Spark.</p>
 </li>
 <li>
 <p>Create a sample.csv file using the following commands. The CSV file is required for loading data into CarbonData.</p>
@@ -181,14 +232,29 @@ EOF
 </li>
 </ul>
 <h2>
-<a id="interactive-analysis-with-spark-shell-version-21" class="anchor" href="#interactive-analysis-with-spark-shell-version-21" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Interactive Analysis with Spark Shell Version 2.1</h2>
+<a id="deployment-modes" class="anchor" href="#deployment-modes" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deployment modes</h2>
+<p>CarbonData can be integrated with Spark and Presto Execution Engines.The below documentation guides on Installing and Configuring with these execution engines.</p>
+<h3>
+<a id="spark" class="anchor" href="#spark" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Spark</h3>
+<p><a href="#installing-and-configuring-carbondata-to-run-locally-with-spark-shell">Installing and Configuring CarbonData to run locally with Spark Shell</a></p>
+<p><a href="#installing-and-configuring-carbondata-on-standalone-spark-cluster">Installing and Configuring CarbonData on Standalone Spark Cluster</a></p>
+<p><a href="#installing-and-configuring-carbondata-on-spark-on-yarn-cluster">Installing and Configuring CarbonData on Spark on YARN Cluster</a></p>
+<h3>
+<a id="presto" class="anchor" href="#presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Presto</h3>
+<p><a href="#installing-and-configuring-carbondata-on-presto">Installing and Configuring CarbonData on Presto</a></p>
+<h2>
+<a id="querying-data" class="anchor" href="#querying-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Querying Data</h2>
+<p><a href="#query-execution-using-carbondata-thrift-server">Query Execution using CarbonData Thrift Server</a></p>
+<h2></h2>
+<h2>
+<a id="installing-and-configuring-carbondata-to-run-locally-with-spark-shell" class="anchor" href="#installing-and-configuring-carbondata-to-run-locally-with-spark-shell" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData to run locally with Spark Shell</h2>
 <p>Apache Spark Shell provides a simple way to learn the API, as well as a powerful tool to analyze data interactively. Please visit <a href="http://spark.apache.org/docs/latest/" target=_blank rel="nofollow">Apache Spark Documentation</a> for more details on Spark shell.</p>
 <h4>
 <a id="basics" class="anchor" href="#basics" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Basics</h4>
 <p>Start Spark shell by running the following command in the Spark directory:</p>
 <pre><code>./bin/spark-shell --jars &lt;carbondata assembly jar path&gt;
 </code></pre>
-<p><strong>NOTE</strong>: Assembly jar will be available after <a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>building CarbonData</a> and can be copied from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code></p>
+<p><strong>NOTE</strong>: Path where packaged release of CarbonData was downloaded or assembly jar will be available after <a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>building CarbonData</a> and can be copied from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code></p>
 <p>In this shell, SparkSession is readily available as <code>spark</code> and Spark context is readily available as <code>sc</code>.</p>
 <p>In order to create a CarbonSession we will have to configure it explicitly in the following manner :</p>
 <ul>
@@ -203,7 +269,7 @@ import org.apache.spark.sql.CarbonSession._
 <pre><code>val carbon = SparkSession.builder().config(sc.getConf)
              .getOrCreateCarbonSession("&lt;hdfs store path&gt;")
 </code></pre>
-<p><strong>NOTE</strong>: By default metastore location is pointed to <code>../carbon.metastore</code>, user can provide own metastore location to CarbonSession like <code>SparkSession.builder().config(sc.getConf) .getOrCreateCarbonSession("&lt;hdfs store path&gt;", "&lt;local metastore path&gt;")</code></p>
+<p><strong>NOTE</strong>: By default metastore location points to <code>../carbon.metastore</code>, user can provide own metastore location to CarbonSession like <code>SparkSession.builder().config(sc.getConf) .getOrCreateCarbonSession("&lt;hdfs store path&gt;", "&lt;local metastore path&gt;")</code></p>
 <h4>
 <a id="executing-queries" class="anchor" href="#executing-queries" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Executing Queries</h4>
 <h6>
@@ -222,7 +288,7 @@ import org.apache.spark.sql.CarbonSession._
                   INTO TABLE test_table")
 </code></pre>
 <p><strong>NOTE</strong>: Please provide the real file path of <code>sample.csv</code> for the above script.
-If you get "tablestatus.lock" issue, please refer to <a href="troubleshooting.html">troubleshooting</a></p>
+If you get "tablestatus.lock" issue, please refer to <a href="faq.html">FAQ</a></p>
 <h6>
 <a id="query-data-from-a-table" class="anchor" href="#query-data-from-a-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Query Data from a Table</h6>
 <pre><code>scala&gt;carbon.sql("SELECT * FROM test_table").show()
@@ -231,6 +297,398 @@ scala&gt;carbon.sql("SELECT city, avg(age), sum(age)
                   FROM test_table
                   GROUP BY city").show()
 </code></pre>
+<h2>
+<a id="installing-and-configuring-carbondata-on-standalone-spark-cluster" class="anchor" href="#installing-and-configuring-carbondata-on-standalone-spark-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Standalone Spark Cluster</h2>
+<h3>
+<a id="prerequisites-1" class="anchor" href="#prerequisites-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h3>
+<ul>
+<li>Hadoop HDFS and Yarn should be installed and running.</li>
+<li>Spark should be installed and running on all the cluster nodes.</li>
+<li>CarbonData user should have permission to access HDFS.</li>
+</ul>
+<h3>
+<a id="procedure" class="anchor" href="#procedure" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Procedure</h3>
+<ol>
+<li>
+<p><a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>Build the CarbonData</a> project and get the assembly jar from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code>.</p>
+</li>
+<li>
+<p>Copy <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code> to <code>$SPARK_HOME/carbonlib</code> folder.</p>
+<p><strong>NOTE</strong>: Create the carbonlib folder if it does not exist inside <code>$SPARK_HOME</code> path.</p>
+</li>
+<li>
+<p>Add the carbonlib folder path in the Spark classpath. (Edit <code>$SPARK_HOME/conf/spark-env.sh</code> file and modify the value of <code>SPARK_CLASSPATH</code> by appending <code>$SPARK_HOME/carbonlib/*</code> to the existing value)</p>
+</li>
+<li>
+<p>Copy the <code>./conf/carbon.properties.template</code> file from CarbonData repository to <code>$SPARK_HOME/conf/</code> folder and rename the file to <code>carbon.properties</code>.</p>
+</li>
+<li>
+<p>Repeat Step 2 to Step 5 in all the nodes of the cluster.</p>
+</li>
+<li>
+<p>In Spark node[master], configure the properties mentioned in the following table in <code>$SPARK_HOME/conf/spark-defaults.conf</code> file.</p>
+</li>
+</ol>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Value</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>spark.driver.extraJavaOptions</td>
+<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
+<td>A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.</td>
+</tr>
+<tr>
+<td>spark.executor.extraJavaOptions</td>
+<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
+<td>A string of extra JVM options to pass to executors. For instance, GC settings or other logging. <strong>NOTE</strong>: You can enter multiple values separated by space.</td>
+</tr>
+</tbody>
+</table>
+<ol>
+<li>Add the following properties in <code>$SPARK_HOME/conf/carbon.properties</code> file:</li>
+</ol>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Required</th>
+<th>Description</th>
+<th>Example</th>
+<th>Remark</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>carbon.storelocation</td>
+<td>NO</td>
+<td>Location where data CarbonData will create the store and write the data in its own format. If not specified then it takes spark.sql.warehouse.dir path.</td>
+<td>hdfs://HOSTNAME:PORT/Opt/CarbonStore</td>
+<td>Propose to set HDFS directory</td>
+</tr>
+</tbody>
+</table>
+<ol>
+<li>Verify the installation. For example:</li>
+</ol>
+<pre><code>./spark-shell --master spark://HOSTNAME:PORT --total-executor-cores 2
+--executor-memory 2G
+</code></pre>
+<p><strong>NOTE</strong>: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.</p>
+<h2>
+<a id="installing-and-configuring-carbondata-on-spark-on-yarn-cluster" class="anchor" href="#installing-and-configuring-carbondata-on-spark-on-yarn-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Spark on YARN Cluster</h2>
+<p>This section provides the procedure to install CarbonData on "Spark on YARN" cluster.</p>
+<h3>
+<a id="prerequisites-2" class="anchor" href="#prerequisites-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h3>
+<ul>
+<li>Hadoop HDFS and Yarn should be installed and running.</li>
+<li>Spark should be installed and running in all the clients.</li>
+<li>CarbonData user should have permission to access HDFS.</li>
+</ul>
+<h3>
+<a id="procedure-1" class="anchor" href="#procedure-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Procedure</h3>
+<p>The following steps are only for Driver Nodes. (Driver nodes are the one which starts the spark context.)</p>
+<ol>
+<li>
+<p><a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>Build the CarbonData</a> project and get the assembly jar from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code> and copy to <code>$SPARK_HOME/carbonlib</code> folder.</p>
+<p><strong>NOTE</strong>: Create the carbonlib folder if it does not exists inside <code>$SPARK_HOME</code> path.</p>
+</li>
+<li>
+<p>Copy the <code>./conf/carbon.properties.template</code> file from CarbonData repository to <code>$SPARK_HOME/conf/</code> folder and rename the file to <code>carbon.properties</code>.</p>
+</li>
+<li>
+<p>Create <code>tar.gz</code> file of carbonlib folder and move it inside the carbonlib folder.</p>
+</li>
+</ol>
+<pre><code>cd $SPARK_HOME
+tar -zcvf carbondata.tar.gz carbonlib/
+mv carbondata.tar.gz carbonlib/
+</code></pre>
+<ol>
+<li>Configure the properties mentioned in the following table in <code>$SPARK_HOME/conf/spark-defaults.conf</code> file.</li>
+</ol>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Description</th>
+<th>Value</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>spark.master</td>
+<td>Set this value to run the Spark in yarn cluster mode.</td>
+<td>Set yarn-client to run the Spark in yarn cluster mode.</td>
+</tr>
+<tr>
+<td>spark.yarn.dist.files</td>
+<td>Comma-separated list of files to be placed in the working directory of each executor.</td>
+<td><code>$SPARK_HOME/conf/carbon.properties</code></td>
+</tr>
+<tr>
+<td>spark.yarn.dist.archives</td>
+<td>Comma-separated list of archives to be extracted into the working directory of each executor.</td>
+<td><code>$SPARK_HOME/carbonlib/carbondata.tar.gz</code></td>
+</tr>
+<tr>
+<td>spark.executor.extraJavaOptions</td>
+<td>A string of extra JVM options to pass to executors. For instance  <strong>NOTE</strong>: You can enter multiple values separated by space.</td>
+<td><code>-Dcarbon.properties.filepath = carbon.properties</code></td>
+</tr>
+<tr>
+<td>spark.executor.extraClassPath</td>
+<td>Extra classpath entries to prepend to the classpath of executors. <strong>NOTE</strong>: If SPARK_CLASSPATH is defined in spark-env.sh, then comment it and append the values in below parameter spark.driver.extraClassPath</td>
+<td><code>carbondata.tar.gz/carbonlib/*</code></td>
+</tr>
+<tr>
+<td>spark.driver.extraClassPath</td>
+<td>Extra classpath entries to prepend to the classpath of the driver. <strong>NOTE</strong>: If SPARK_CLASSPATH is defined in spark-env.sh, then comment it and append the value in below parameter spark.driver.extraClassPath.</td>
+<td><code>$SPARK_HOME/carbonlib/*</code></td>
+</tr>
+<tr>
+<td>spark.driver.extraJavaOptions</td>
+<td>A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.</td>
+<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
+</tr>
+</tbody>
+</table>
+<ol>
+<li>Add the following properties in <code>$SPARK_HOME/conf/carbon.properties</code>:</li>
+</ol>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Required</th>
+<th>Description</th>
+<th>Example</th>
+<th>Default Value</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>carbon.storelocation</td>
+<td>NO</td>
+<td>Location where CarbonData will create the store and write the data in its own format. If not specified then it takes spark.sql.warehouse.dir path.</td>
+<td>hdfs://HOSTNAME:PORT/Opt/CarbonStore</td>
+<td>Propose to set HDFS directory</td>
+</tr>
+</tbody>
+</table>
+<ol>
+<li>Verify the installation.</li>
+</ol>
+<pre><code> ./bin/spark-shell --master yarn-client --driver-memory 1g
+ --executor-cores 2 --executor-memory 2G
+</code></pre>
+<p><strong>NOTE</strong>: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.</p>
+<h2>
+<a id="query-execution-using-carbondata-thrift-server" class="anchor" href="#query-execution-using-carbondata-thrift-server" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Query Execution Using CarbonData Thrift Server</h2>
+<h3>
+<a id="starting-carbondata-thrift-server" class="anchor" href="#starting-carbondata-thrift-server" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Starting CarbonData Thrift Server.</h3>
+<p>a. cd <code>$SPARK_HOME</code></p>
+<p>b. Run the following command to start the CarbonData thrift server.</p>
+<pre><code>./bin/spark-submit
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer
+$SPARK_HOME/carbonlib/$CARBON_ASSEMBLY_JAR &lt;carbon_store_path&gt;
+</code></pre>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Example</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>CARBON_ASSEMBLY_JAR</td>
+<td>CarbonData assembly jar name present in the <code>$SPARK_HOME/carbonlib/</code> folder.</td>
+<td>carbondata_2.xx-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar</td>
+</tr>
+<tr>
+<td>carbon_store_path</td>
+<td>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. If not specified then it takes spark.sql.warehouse.dir path.</td>
+<td><code>hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store</code></td>
+</tr>
+</tbody>
+</table>
+<p><strong>NOTE</strong>: From Spark 1.6, by default the Thrift server runs in multi-session mode. Which means each JDBC/ODBC connection owns a copy of their own SQL configuration and temporary function registry. Cached tables are still shared though. If you prefer to run the Thrift server in single-session mode and share all SQL configuration and temporary function registry, please set option <code>spark.sql.hive.thriftServer.singleSession</code> to <code>true</code>. You may either add this option to <code>spark-defaults.conf</code>, or pass it to <code>spark-submit.sh</code> via <code>--conf</code>:</p>
+<pre><code>./bin/spark-submit
+--conf spark.sql.hive.thriftServer.singleSession=true
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer
+$SPARK_HOME/carbonlib/$CARBON_ASSEMBLY_JAR &lt;carbon_store_path&gt;
+</code></pre>
+<p><strong>But</strong> in single-session mode, if one user changes the database from one connection, the database of the other connections will be changed too.</p>
+<p><strong>Examples</strong></p>
+<ul>
+<li>Start with default memory and executors.</li>
+</ul>
+<pre><code>./bin/spark-submit
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer 
+$SPARK_HOME/carbonlib
+/carbondata_2.xx-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar
+hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
+</code></pre>
+<ul>
+<li>Start with Fixed executors and resources.</li>
+</ul>
+<pre><code>./bin/spark-submit
+--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://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
+</code></pre>
+<h3>
+<a id="connecting-to-carbondata-thrift-server-using-beeline" class="anchor" href="#connecting-to-carbondata-thrift-server-using-beeline" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Connecting to CarbonData Thrift Server Using Beeline.</h3>
+<pre><code>     cd $SPARK_HOME
+     ./sbin/start-thriftserver.sh
+     ./bin/beeline -u jdbc:hive2://&lt;thriftserver_host&gt;:port
+
+     Example
+     ./bin/beeline -u jdbc:hive2://10.10.10.10:10000
+</code></pre>
+<h2>
+<a id="installing-and-configuring-carbondata-on-presto" class="anchor" href="#installing-and-configuring-carbondata-on-presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Presto</h2>
+<ul>
+<li>
+<h3>
+<a id="installing-presto" class="anchor" href="#installing-presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing Presto</h3>
+</li>
+</ul>
+<ol>
+<li>
+<p>Download the 0.187 version of Presto using:
+<code>wget https://repo1.maven.org/maven2/com/facebook/presto/presto-server/0.187/presto-server-0.187.tar.gz</code></p>
+</li>
+<li>
+<p>Extract Presto tar file: <code>tar zxvf presto-server-0.187.tar.gz</code>.</p>
+</li>
+<li>
+<p>Download the Presto CLI for the coordinator and name it presto.</p>
+</li>
+</ol>
+<pre><code>  wget https://repo1.maven.org/maven2/com/facebook/presto/presto-cli/0.187/presto-cli-0.187-executable.jar
+
+  mv presto-cli-0.187-executable.jar presto
+
+  chmod +x presto
+</code></pre>
+<h3>
+<a id="create-configuration-files" class="anchor" href="#create-configuration-files" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Configuration Files</h3>
+<ol>
+<li>
+<p>Create <code>etc</code> folder in presto-server-0.187 directory.</p>
+</li>
+<li>
+<p>Create <code>config.properties</code>, <code>jvm.config</code>, <code>log.properties</code>, and <code>node.properties</code> files.</p>
+</li>
+<li>
+<p>Install uuid to generate a node.id.</p>
+<pre><code>sudo apt-get install uuid
+
+uuid
+</code></pre>
+</li>
+</ol>
+<h5>
+<a id="contents-of-your-nodeproperties-file" class="anchor" href="#contents-of-your-nodeproperties-file" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contents of your node.properties file</h5>
+<pre><code>node.environment=production
+node.id=&lt;generated uuid&gt;
+node.data-dir=/home/ubuntu/data
+</code></pre>
+<h5>
+<a id="contents-of-your-jvmconfig-file" class="anchor" href="#contents-of-your-jvmconfig-file" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contents of your jvm.config file</h5>
+<pre><code>-server
+-Xmx16G
+-XX:+UseG1GC
+-XX:G1HeapRegionSize=32M
+-XX:+UseGCOverheadLimit
+-XX:+ExplicitGCInvokesConcurrent
+-XX:+HeapDumpOnOutOfMemoryError
+-XX:OnOutOfMemoryError=kill -9 %p
+</code></pre>
+<h5>
+<a id="contents-of-your-logproperties-file" class="anchor" href="#contents-of-your-logproperties-file" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contents of your log.properties file</h5>
+<pre><code>com.facebook.presto=INFO
+</code></pre>
+<p>The default minimum level is <code>INFO</code>. There are four levels: <code>DEBUG</code>, <code>INFO</code>, <code>WARN</code> and <code>ERROR</code>.</p>
+<h3>
+<a id="coordinator-configurations" class="anchor" href="#coordinator-configurations" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Coordinator Configurations</h3>
+<h5>
+<a id="contents-of-your-configproperties" class="anchor" href="#contents-of-your-configproperties" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contents of your config.properties</h5>
+<pre><code>coordinator=true
+node-scheduler.include-coordinator=false
+http-server.http.port=8086
+query.max-memory=50GB
+query.max-memory-per-node=2GB
+discovery-server.enabled=true
+discovery.uri=&lt;coordinator_ip&gt;:8086
+</code></pre>
+<p>The options <code>node-scheduler.include-coordinator=false</code> and <code>coordinator=true</code> indicate that the node is the coordinator and tells the coordinator not to do any of the computation work itself and to use the workers.</p>
+<p><strong>Note</strong>: It is recommended to set <code>query.max-memory-per-node</code> to half of the JVM config max memory, though the workload is highly concurrent, lower value for <code>query.max-memory-per-node</code> is to be used.</p>
+<p>Also relation between below two configuration-properties should be like:
+If, <code>query.max-memory-per-node=30GB</code>
+Then, <code>query.max-memory=&lt;30GB * number of nodes&gt;</code>.</p>
+<h3>
+<a id="worker-configurations" class="anchor" href="#worker-configurations" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Worker Configurations</h3>
+<h5>
+<a id="contents-of-your-configproperties-1" class="anchor" href="#contents-of-your-configproperties-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contents of your config.properties</h5>
+<pre><code>coordinator=false
+http-server.http.port=8086
+query.max-memory=50GB
+query.max-memory-per-node=2GB
+discovery.uri=&lt;coordinator_ip&gt;:8086
+</code></pre>
+<p><strong>Note</strong>: <code>jvm.config</code> and <code>node.properties</code> files are same for all the nodes (worker + coordinator). All the nodes should have different <code>node.id</code>.(generated by uuid command).</p>
+<h3>
+<a id="catalog-configurations" class="anchor" href="#catalog-configurations" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Catalog Configurations</h3>
+<ol>
+<li>Create a folder named <code>catalog</code> in etc directory of presto on all the nodes of the cluster including the coordinator.</li>
+</ol>
+<h5>
+<a id="configuring-carbondata-in-presto" class="anchor" href="#configuring-carbondata-in-presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configuring Carbondata in Presto</h5>
+<ol>
+<li>Create a file named <code>carbondata.properties</code> in the <code>catalog</code> folder and set the required properties on all the nodes.</li>
+</ol>
+<h3>
+<a id="add-plugins" class="anchor" href="#add-plugins" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Add Plugins</h3>
+<ol>
+<li>Create a directory named <code>carbondata</code> in plugin directory of presto.</li>
+<li>Copy <code>carbondata</code> jars to <code>plugin/carbondata</code> directory on all nodes.</li>
+</ol>
+<h3>
+<a id="start-presto-server-on-all-nodes" class="anchor" href="#start-presto-server-on-all-nodes" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Start Presto Server on all nodes</h3>
+<pre><code>./presto-server-0.187/bin/launcher start
+</code></pre>
+<p>To run it as a background process.</p>
+<pre><code>./presto-server-0.187/bin/launcher run
+</code></pre>
+<p>To run it in foreground.</p>
+<h3>
+<a id="start-presto-cli" class="anchor" href="#start-presto-cli" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Start Presto CLI</h3>
+<pre><code>./presto
+</code></pre>
+<p>To connect to carbondata catalog use the following command:</p>
+<pre><code>./presto --server &lt;coordinator_ip&gt;:8086 --catalog carbondata --schema &lt;schema_name&gt;
+</code></pre>
+<p>Execute the following command to ensure the workers are connected.</p>
+<pre><code>select * from system.runtime.nodes;
+</code></pre>
+<p>Now you can use the Presto CLI on the coordinator to query data sources in the catalog using the Presto workers.</p>
+<p><strong>Note :</strong> Create Tables and data loads should be done before executing queries as we can not create carbon table from this interface.</p>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__quickstart').addClass('selected'); });
+</script>
 </div>
 </div>
 </div>
@@ -246,4 +704,4 @@ scala&gt;carbon.sql("SELECT city, avg(age), sum(age)
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/release-guide.html
----------------------------------------------------------------------
diff --git a/content/release-guide.html b/content/release-guide.html
index 0a66cce..fb51368 100644
--- a/content/release-guide.html
+++ b/content/release-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -51,32 +52,23 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
-                                   target="_blank">Apache CarbonData 1.2.0</a></li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
                             <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
-                                   target="_blank">Apache CarbonData 1.1.1</a></li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
-                                   target="_blank">Apache CarbonData 1.1.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
-                                   target="_blank">Apache CarbonData 1.0.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
-                                   target="_blank">Apache CarbonData 0.2.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
-                                   target="_blank">Apache CarbonData 0.1.1</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
-                                   target="_blank">Apache CarbonData 0.1.0</a></li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
                             <li>
                                 <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -161,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -508,6 +550,10 @@ contributions.</p>
 <li>Release announced on the Incubator's general@ mailing list.</li>
 <li>Completion declared on the dev@ mailing list.</li>
 </ol>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__release').addClass('selected'); });
+</script>
 </div>
 </div>
 </div>
@@ -523,4 +569,4 @@ contributions.</p>
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/s3-guide.html
----------------------------------------------------------------------
diff --git a/content/s3-guide.html b/content/s3-guide.html
index d9edeaf..9042b19 100644
--- a/content/s3-guide.html
+++ b/content/s3-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,14 +153,65 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
                             <div class="row">
                                 <div class="col-sm-12  col-md-12">
                                     <div>
-<p>#S3 Guide (Alpha Feature 1.4.1)</p>
+<h1>
+<a id="s3-guide-alpha-feature-141" class="anchor" href="#s3-guide-alpha-feature-141" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>S3 Guide (Alpha Feature 1.4.1)</h1>
 <p>Object storage is the recommended storage format in cloud as it can support storing large data
 files. S3 APIs are widely used for accessing object stores. This can be
 used to store or retrieve data on Amazon cloud, Huawei Cloud(OBS) or on any other object
@@ -168,7 +220,8 @@ Storing data in cloud is advantageous as there are no restrictions on the size o
 data and the data can be accessed from anywhere at any time.
 Carbondata can support any Object Storage that conforms to Amazon S3 API.
 Carbondata relies on Hadoop provided S3 filesystem APIs to access Object stores.</p>
-<p>#Writing to Object Storage</p>
+<h1>
+<a id="writing-to-object-storage" class="anchor" href="#writing-to-object-storage" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Writing to Object Storage</h1>
 <p>To store carbondata files onto Object Store, <code>carbon.storelocation</code> property will have
 to be configured with Object Store path in CarbonProperties file.</p>
 <p>For example:</p>
@@ -180,8 +233,9 @@ table DDL command.</p>
 <p>For example:</p>
 <pre><code>CREATE TABLE IF NOT EXISTS db1.table1(col1 string, col2 int) STORED AS carbondata LOCATION 's3a://mybucket/carbonstore'
 </code></pre>
-<p>For more details on create table, Refer <a href="./data-management-on-carbondata.html#create-table">data-management-on-carbondata</a></p>
-<p>#Authentication</p>
+<p>For more details on create table, Refer <a href="ddl-of-carbondata.html#create-table">DDL of CarbonData</a></p>
+<h1>
+<a id="authentication" class="anchor" href="#authentication" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Authentication</h1>
 <p>Authentication properties will have to be configured to store the carbondata files on to S3 location.</p>
 <p>Authentication properties can be set in any of the following ways:</p>
 <ol>
@@ -211,16 +265,19 @@ spark.hadoop.fs.s3a.access.key=456
 <pre><code>sparkSession.sparkContext.hadoopConfiguration.set("fs.s3a.secret.key", "123")
 sparkSession.sparkContext.hadoopConfiguration.set("fs.s3a.access.key","456")
 </code></pre>
-<p>#Recommendations</p>
+<h1>
+<a id="recommendations" class="anchor" href="#recommendations" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Recommendations</h1>
 <ol>
 <li>Object Storage like S3 does not support file leasing mechanism(supported by HDFS) that is
 required to take locks which ensure consistency between concurrent operations therefore, it is
-recommended to set the configurable lock path property(<a href="https://github.com/apache/carbondata/blob/master/docs/configuration-parameters.html#miscellaneous-configuration">carbon.lock.path</a>)
+recommended to set the configurable lock path property(<a href="./configuration-parameters.html#system-configuration">carbon.lock.path</a>)
 to a HDFS directory.</li>
-<li>Concurrent data manipulation operations are not supported. Object stores follow eventual
-consistency semantics, i.e., any put request might take some time to reflect when trying to list
-.This behaviour causes not to ensure the data read is always consistent or latest.</li>
+<li>Concurrent data manipulation operations are not supported. Object stores follow eventual consistency semantics, i.e., any put request might take some time to reflect when trying to list. This behaviour causes the data read is always not consistent or not the latest.</li>
 </ol>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__s3').addClass('selected'); });
+</script>
 </div>
 </div>
 </div>
@@ -236,4 +293,4 @@ consistency semantics, i.e., any put request might take some time to reflect whe
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>


[22/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/troubleshooting.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/troubleshooting.html b/src/main/webapp/troubleshooting.html
deleted file mode 100644
index 7e0cd60..0000000
--- a/src/main/webapp/troubleshooting.html
+++ /dev/null
@@ -1,423 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
-                                   target="_blank">Apache CarbonData 1.4.1</a></li>
-							<li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
-                                   target="_blank">Apache CarbonData 1.4.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
-                                   target="_blank">Apache CarbonData 1.3.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
-                                   target="_blank">Apache CarbonData 1.3.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="documentation.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search" target="_blank">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input"  placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="verticalnavbar">
-                <nav class="b-sticky-nav">
-                    <div class="nav-scroller">
-                        <div class="nav__inner">
-                            <a class="b-nav__intro nav__item" href="./videogallery.html">introduction</a>
-                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
-
-                            <div class="nav__item nav__item__with__subs">
-                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
-                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
-                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
-                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
-                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
-                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
-                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
-                            </div>
-
-                            <div class="nav__item nav__item__with__subs">
-                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
-                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
-                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
-                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
-                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
-                            </div>
-
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
-                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
-                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
-                            <a class="b-nav__trouble nav__item" href="./troubleshooting.html">Troubleshooting</a>
-                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
-                            <a class="b-nav__contri nav__item" href="./videogallery.html">Contribute</a>
-                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
-                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
-                        </div>
-                    </div>
-                    <div class="navindicator">
-                        <div class="b-nav__intro navindicator__item"></div>
-                        <div class="b-nav__quickstart navindicator__item"></div>
-                        <div class="b-nav__uses navindicator__item"></div>
-                        <div class="b-nav__docs navindicator__item"></div>
-                        <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
-                        <div class="b-nav__api navindicator__item"></div>
-                        <div class="b-nav__perf navindicator__item"></div>
-                        <div class="b-nav__trouble navindicator__item"></div>
-                        <div class="b-nav__faq navindicator__item"></div>
-                        <div class="b-nav__contri navindicator__item"></div>
-                        <div class="b-nav__security navindicator__item"></div>
-                    </div>
-                </nav>
-            </div>
-            <div class="mdcontent">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="row">
-                                <div class="col-sm-12  col-md-12">
-                                    <div>
-<h1>
-<a id="troubleshooting" class="anchor" href="#troubleshooting" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Troubleshooting</h1>
-<p>This tutorial is designed to provide troubleshooting for end users and developers
-who are building, deploying, and using CarbonData.</p>
-<h2>
-<a id="when-loading-data-gets-tablestatuslock-issues" class="anchor" href="#when-loading-data-gets-tablestatuslock-issues" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>When loading data, gets tablestatus.lock issues:</h2>
-<p><strong>Symptom</strong></p>
-<pre><code>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.&lt;init&gt;(FileOutputStream.java:213)
-	at java.io.FileOutputStream.&lt;init&gt;(FileOutputStream.java:101)
-</code></pre>
-<p><strong>Possible Cause</strong>
-If you use <code>&lt;hdfs path&gt;</code> as store path when creating carbonsession, may get the errors,because the default is LOCALLOCK.</p>
-<p><strong>Procedure</strong>
-Before creating carbonsession, sets as below:</p>
-<pre><code>import org.apache.carbondata.core.util.CarbonProperties
-import org.apache.carbondata.core.constants.CarbonCommonConstants
-CarbonProperties.getInstance().addProperty(CarbonCommonConstants.LOCK_TYPE, "HDFSLOCK")
-</code></pre>
-<h2>
-<a id="failed-to-load-thrift-libraries" class="anchor" href="#failed-to-load-thrift-libraries" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to load thrift libraries</h2>
-<p><strong>Symptom</strong></p>
-<p>Thrift throws following exception :</p>
-<pre><code>thrift: error while loading shared libraries:
-libthriftc.so.0: cannot open shared object file: No such file or directory
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The complete path to the directory containing the libraries is not configured correctly.</p>
-<p><strong>Procedure</strong></p>
-<p>Follow the Apache thrift docs at <a href="https://thrift.apache.org/docs/install" target=_blank rel="nofollow">https://thrift.apache.org/docs/install</a> to install thrift correctly.</p>
-<h2>
-<a id="failed-to-launch-the-spark-shell" class="anchor" href="#failed-to-launch-the-spark-shell" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to launch the Spark Shell</h2>
-<p><strong>Symptom</strong></p>
-<p>The shell prompts the following error :</p>
-<pre><code>org.apache.spark.sql.CarbonContext$$anon$$apache$spark$sql$catalyst$analysis
-$OverrideCatalog$_setter_$org$apache$spark$sql$catalyst$analysis
-$OverrideCatalog$$overrides_$e
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The Spark Version and the selected Spark Profile do not match.</p>
-<p><strong>Procedure</strong></p>
-<ol>
-<li>
-<p>Ensure your spark version and selected profile for spark are correct.</p>
-</li>
-<li>
-<p>Use the following command :</p>
-</li>
-</ol>
-<pre><code>"mvn -Pspark-2.1 -Dspark.version {yourSparkVersion} clean package"
-</code></pre>
-<p>Note :  Refrain from using "mvn clean package" without specifying the profile.</p>
-<h2>
-<a id="failed-to-execute-load-query-on-cluster" class="anchor" href="#failed-to-execute-load-query-on-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute load query on cluster.</h2>
-<p><strong>Symptom</strong></p>
-<p>Load query failed with the following exception:</p>
-<pre><code>Dictionary file is locked for updation.
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The carbon.properties file is not identical in all the nodes of the cluster.</p>
-<p><strong>Procedure</strong></p>
-<p>Follow the steps to ensure the carbon.properties file is consistent across all the nodes:</p>
-<ol>
-<li>
-<p>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.</p>
-</li>
-<li>
-<p>For the changes to take effect, restart the Spark cluster.</p>
-</li>
-</ol>
-<h2>
-<a id="failed-to-execute-insert-query-on-cluster" class="anchor" href="#failed-to-execute-insert-query-on-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute insert query on cluster.</h2>
-<p><strong>Symptom</strong></p>
-<p>Load query failed with the following exception:</p>
-<pre><code>Dictionary file is locked for updation.
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The carbon.properties file is not identical in all the nodes of the cluster.</p>
-<p><strong>Procedure</strong></p>
-<p>Follow the steps to ensure the carbon.properties file is consistent across all the nodes:</p>
-<ol>
-<li>
-<p>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.</p>
-</li>
-<li>
-<p>For the changes to take effect, restart the Spark cluster.</p>
-</li>
-</ol>
-<h2>
-<a id="failed-to-connect-to-hiveuser-with-thrift" class="anchor" href="#failed-to-connect-to-hiveuser-with-thrift" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to connect to hiveuser with thrift</h2>
-<p><strong>Symptom</strong></p>
-<p>We get the following exception :</p>
-<pre><code>Cannot connect to hiveuser.
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The external process does not have permission to access.</p>
-<p><strong>Procedure</strong></p>
-<p>Ensure that the Hiveuser in mysql must allow its access to the external processes.</p>
-<h2>
-<a id="failed-to-read-the-metastore-db-during-table-creation" class="anchor" href="#failed-to-read-the-metastore-db-during-table-creation" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to read the metastore db during table creation.</h2>
-<p><strong>Symptom</strong></p>
-<p>We get the following exception on trying to connect :</p>
-<pre><code>Cannot read the metastore db
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The metastore db is dysfunctional.</p>
-<p><strong>Procedure</strong></p>
-<p>Remove the metastore db from the carbon.metastore in the Spark Directory.</p>
-<h2>
-<a id="failed-to-load-data-on-the-cluster" class="anchor" href="#failed-to-load-data-on-the-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to load data on the cluster</h2>
-<p><strong>Symptom</strong></p>
-<p>Data loading fails with the following exception :</p>
-<pre><code>Data Load failure exception
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The following issue can cause the failure :</p>
-<ol>
-<li>
-<p>The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.</p>
-</li>
-<li>
-<p>Path to hdfs ddl is not configured correctly in the carbon.properties.</p>
-</li>
-</ol>
-<p><strong>Procedure</strong></p>
-<p>Follow the steps to ensure the following configuration files are consistent across all the nodes:</p>
-<ol>
-<li>
-<p>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.</p>
-<p>Note : Set the path to hdfs ddl in carbon.properties in the master node.</p>
-</li>
-<li>
-<p>For the changes to take effect, restart the Spark cluster.</p>
-</li>
-</ol>
-<h2>
-<a id="failed-to-insert-data-on-the-cluster" class="anchor" href="#failed-to-insert-data-on-the-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to insert data on the cluster</h2>
-<p><strong>Symptom</strong></p>
-<p>Insertion fails with the following exception :</p>
-<pre><code>Data Load failure exception
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>The following issue can cause the failure :</p>
-<ol>
-<li>
-<p>The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.</p>
-</li>
-<li>
-<p>Path to hdfs ddl is not configured correctly in the carbon.properties.</p>
-</li>
-</ol>
-<p><strong>Procedure</strong></p>
-<p>Follow the steps to ensure the following configuration files are consistent across all the nodes:</p>
-<ol>
-<li>
-<p>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.</p>
-<p>Note : Set the path to hdfs ddl in carbon.properties in the master node.</p>
-</li>
-<li>
-<p>For the changes to take effect, restart the Spark cluster.</p>
-</li>
-</ol>
-<h2>
-<a id="failed-to-execute-concurrent-operationsloadinsertupdate-on-table-by-multiple-workers" class="anchor" href="#failed-to-execute-concurrent-operationsloadinsertupdate-on-table-by-multiple-workers" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute Concurrent Operations(Load,Insert,Update) on table by multiple workers.</h2>
-<p><strong>Symptom</strong></p>
-<p>Execution fails with the following exception :</p>
-<pre><code>Table is locked for updation.
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>Concurrency not supported.</p>
-<p><strong>Procedure</strong></p>
-<p>Worker must wait for the query execution to complete and the table to release the lock for another query execution to succeed.</p>
-<h2>
-<a id="failed-to-create-a-table-with-a-single-numeric-column" class="anchor" href="#failed-to-create-a-table-with-a-single-numeric-column" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to create a table with a single numeric column.</h2>
-<p><strong>Symptom</strong></p>
-<p>Execution fails with the following exception :</p>
-<pre><code>Table creation fails.
-</code></pre>
-<p><strong>Possible Cause</strong></p>
-<p>Behaviour not supported.</p>
-<p><strong>Procedure</strong></p>
-<p>A single column that can be considered as dimension is mandatory for table creation.</p>
-<script>
-// Show selected style on nav item
-$(function() { $('.b-nav__trouble').addClass('selected'); });
-</script>
-</div>
-</div>
-</div>
-</div>
-<div class="doc-footer">
-    <a href="#top" class="scroll-top">Top</a>
-</div>
-</div>
-</section>
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/usecases.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/usecases.html b/src/main/webapp/usecases.html
new file mode 100644
index 0000000..cb309dd
--- /dev/null
+++ b/src/main/webapp/usecases.html
@@ -0,0 +1,619 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div><h1>
+<a id="use-cases" class="anchor" href="#use-cases" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Use Cases</h1>
+<p>CarbonData is useful in various analytical work loads.Some of the most typical usecases where CarbonData is being used is documented here.</p>
+<p>CarbonData is used for but not limited to</p>
+<ul>
+<li>
+<h3>
+<a id="bank" class="anchor" href="#bank" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Bank</h3>
+<ul>
+<li>fraud detection analysis</li>
+<li>risk profile analysis</li>
+<li>As a zip table to update the daily balance of customers</li>
+</ul>
+</li>
+<li>
+<h3>
+<a id="telecom" class="anchor" href="#telecom" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Telecom</h3>
+<ul>
+<li>Detection of signal anamolies for VIP customers for providing improved customer experience</li>
+<li>Analysis of MR,CHR records of GSM data to determine the tower load at a particular time period and rebalance the tower configuration</li>
+<li>Analysis of access sites, video, screen size, streaming bandwidth, quality to determine the network quality,routing configuration</li>
+</ul>
+</li>
+<li>
+<h3>
+<a id="webinternet" class="anchor" href="#webinternet" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Web/Internet</h3>
+<ul>
+<li>Analysis of page or video being accessed,server loads, streaming quality, screen size</li>
+</ul>
+</li>
+<li>
+<h3>
+<a id="smart-city" class="anchor" href="#smart-city" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Smart City</h3>
+<ul>
+<li>Vehicle tracking analysis</li>
+<li>Unusual behaviour analysis</li>
+</ul>
+</li>
+</ul>
+<p>These use cases can be broadly classified into below categories:</p>
+<ul>
+<li>Full scan/Detailed/Interactive queries</li>
+<li>Aggregation/OLAP BI queries</li>
+<li>Real time Ingestion(Streaming) and queries</li>
+</ul>
+<h2>
+<a id="detailed-queries-in-the-telecom-scenario" class="anchor" href="#detailed-queries-in-the-telecom-scenario" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Detailed Queries in the Telecom scenario</h2>
+<h3>
+<a id="scenario" class="anchor" href="#scenario" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Scenario</h3>
+<p>User wants to analyse all the CHR(Call History Record) and MR(Measurement Records) of the mobile subscribers in order to identify the service failures within 10 secs.Also user wants to run machine learning models on the data to fairly estimate the reasons and time of probable failures and take action ahead to meet the SLA(Service Level Agreements) of VIP customers.</p>
+<h3>
+<a id="challenges" class="anchor" href="#challenges" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Challenges</h3>
+<ul>
+<li>Data incoming rate might vary based on the user concentration at a particular period of time.Hence higher data load speeds are required</li>
+<li>Cluster needs to be well utilised and share the cluster among various applications for better resource consumption and savings</li>
+<li>Queries needs to be interactive.ie., the queries fetch small data and need to be returned in seconds</li>
+<li>Data Loaded into the system every few minutes.</li>
+</ul>
+<h3>
+<a id="solution" class="anchor" href="#solution" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Solution</h3>
+<p>Setup a Hadoop + Spark + CarbonData cluster managed by YARN.</p>
+<p>Proposed the following configurations for CarbonData.(These tunings were proposed before CarbonData introduced SORT_COLUMNS parameter using which the sort order and schema order could be different.)</p>
+<p>Add the frequently used columns to the left of the table definition.Add it in the increasing order of cardinality.It was suggested to keep msisdn,imsi columns in the beginning of the schema.With latest CarbonData, SORT_COLUMNS needs to be configured msisdn,imsi in the beginning.</p>
+<p>Add timestamp column to the right of the schema as it is naturally increasing.</p>
+<p>Create two separate YARN queues for Query and Data Loading.</p>
+<p>Apart from these, the following CarbonData configuration was suggested to be configured in the cluster.</p>
+<table>
+<thead>
+<tr>
+<th>Configuration for</th>
+<th>Parameter</th>
+<th>Value</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>Data Loading</td>
+<td>carbon.graph.rowset.size</td>
+<td>100000</td>
+<td>Based on the size of each row, this determines the memory required during data loading.Higher value leads to increased memory foot print</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.number.of.cores.while.loading</td>
+<td>12</td>
+<td>More cores can improve data loading speed</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.sort.size</td>
+<td>100000</td>
+<td>Number of records to sort at a time.More number of records configured will lead to increased memory foot print</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>table_blocksize</td>
+<td>256</td>
+<td>To efficiently schedule multiple tasks during query</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.sort.intermediate.files.limit</td>
+<td>100</td>
+<td>Increased to 100 as number of cores are more.Can perform merging in backgorund.If less number of files to merge, sort threads would be idle</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.use.local.dir</td>
+<td>TRUE</td>
+<td>yarn application directory will be usually on a single disk.YARN would be configured with multiple disks to be used as temp or to assign randomly to applications.Using the yarn temp directory will allow carbon to use multiple disks and improve IO performance</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.use.multiple.temp.dir</td>
+<td>TRUE</td>
+<td>multiple disks to write sort files will lead to better IO and reduce the IO bottleneck</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.compaction.level.threshold</td>
+<td>6,6</td>
+<td>Since frequent small loads, compacting more segments will give better query results</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.enable.auto.load.merge</td>
+<td>true</td>
+<td>Since data loading is small,auto compacting keeps the number of segments less and also compaction can complete in  time</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.number.of.cores.while.compacting</td>
+<td>4</td>
+<td>Higher number of cores can improve the compaction speed</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.major.compaction.size</td>
+<td>921600</td>
+<td>Sum of several loads to combine into single segment</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="results-achieved" class="anchor" href="#results-achieved" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Results Achieved</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Results</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>Query</td>
+<td>&lt; 3 Sec</td>
+</tr>
+<tr>
+<td>Data Loading Speed</td>
+<td>40 MB/s Per Node</td>
+</tr>
+<tr>
+<td>Concurrent query performance (20 queries)</td>
+<td>&lt; 10 Sec</td>
+</tr>
+</tbody>
+</table>
+<h2>
+<a id="detailed-queries-in-the-smart-city-scenario" class="anchor" href="#detailed-queries-in-the-smart-city-scenario" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Detailed Queries in the Smart City scenario</h2>
+<h3>
+<a id="scenario-1" class="anchor" href="#scenario-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Scenario</h3>
+<p>User wants to analyse the person/vehicle movement and behavior during a certain time period.This output data needs to be joined with a external table for Human details extraction.The query will be run with different time period as filter to identify potential behavior mismatch.</p>
+<h3>
+<a id="challenges-1" class="anchor" href="#challenges-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Challenges</h3>
+<p>Data generated per day is very huge.Data needs to be loaded multiple times per day to accomodate the incoming data size.</p>
+<p>Data Loading done once in 6 hours.</p>
+<h3>
+<a id="solution-1" class="anchor" href="#solution-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Solution</h3>
+<p>Setup a Hadoop + Spark + CarbonData cluster managed by YARN.</p>
+<p>Since data needs to be queried for a time period, it was recommended to keep the time column at the beginning of schema.</p>
+<p>Use table block size as 512MB.</p>
+<p>Use local sort mode.</p>
+<p>Apart from these, the following CarbonData configuration was suggested to be configured in the cluster.</p>
+<p>Use all columns are no-dictionary as the cardinality is high.</p>
+<table>
+<thead>
+<tr>
+<th>Configuration for</th>
+<th>Parameter</th>
+<th>Value</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>Data Loading</td>
+<td>carbon.graph.rowset.size</td>
+<td>100000</td>
+<td>Based on the size of each row, this determines the memory required during data loading.Higher value leads to increased memory foot print</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>enable.unsafe.sort</td>
+<td>TRUE</td>
+<td>Temporary data generated during sort is huge which causes GC bottlenecks.Using unsafe reduces the pressure on GC</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>enable.offheap.sort</td>
+<td>TRUE</td>
+<td>Temporary data generated during sort is huge which causes GC bottlenecks.Using offheap reduces the pressure on GC.offheap can be accessed through java unsafe.hence enable.unsafe.sort needs to be true</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>offheap.sort.chunk.size.in.mb</td>
+<td>128</td>
+<td>Size of memory to allocate for sorting.Can increase this based on the memory available</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.number.of.cores.while.loading</td>
+<td>12</td>
+<td>Higher cores can improve data loading speed</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.sort.size</td>
+<td>100000</td>
+<td>Number of records to sort at a time.More number of records configured will lead to increased memory foot print</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>table_blocksize</td>
+<td>512</td>
+<td>To efficiently schedule multiple tasks during query.This size depends on data scenario.If data is such that the filters would select less number of blocklets to scan, keeping higher number works well.If the number blocklets to scan is more, better to reduce the size as more tasks can be scheduled in parallel.</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.sort.intermediate.files.limit</td>
+<td>100</td>
+<td>Increased to 100 as number of cores are more.Can perform merging in backgorund.If less number of files to merge, sort threads would be idle</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.use.local.dir</td>
+<td>TRUE</td>
+<td>yarn application directory will be usually on a single disk.YARN would be configured with multiple disks to be used as temp or to assign randomly to applications.Using the yarn temp directory will allow carbon to use multiple disks and improve IO performance</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>carbon.use.multiple.temp.dir</td>
+<td>TRUE</td>
+<td>multiple disks to write sort files will lead to better IO and reduce the IO bottleneck</td>
+</tr>
+<tr>
+<td>Data Loading</td>
+<td>sort.inmemory.size.in.mb</td>
+<td>92160</td>
+<td>Memory allocated to do inmemory sorting.When more memory is available in the node, configuring this will retain more sort blocks in memory so that the merge sort is faster due to no/very less IO</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.major.compaction.size</td>
+<td>921600</td>
+<td>Sum of several loads to combine into single segment</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.number.of.cores.while.compacting</td>
+<td>12</td>
+<td>Higher number of cores can improve the compaction speed.Data size is huge.Compaction need to use more threads to speed up the process</td>
+</tr>
+<tr>
+<td>Compaction</td>
+<td>carbon.enable.auto.load.merge</td>
+<td>FALSE</td>
+<td>Doing auto minor compaction is costly process as data size is huge.Perform manual compaction when  the cluster is less loaded</td>
+</tr>
+<tr>
+<td>Query</td>
+<td>carbon.enable.vector.reader</td>
+<td>true</td>
+<td>To fetch results faster, supporting spark vector processing will speed up the query</td>
+</tr>
+<tr>
+<td>Query</td>
+<td>enable.unsafe.in.query.procressing</td>
+<td>true</td>
+<td>Data that needs to be scanned in huge which in turn generates more short lived Java objects.This cause pressure of GC.using unsafe and offheap will reduce the GC overhead</td>
+</tr>
+<tr>
+<td>Query</td>
+<td>use.offheap.in.query.processing</td>
+<td>true</td>
+<td>Data that needs to be scanned in huge which in turn generates more short lived Java objects.This cause pressure of GC.using unsafe and offheap will reduce the GC overhead.offheap can be accessed through java unsafe.hence enable.unsafe.in.query.procressing needs to be true</td>
+</tr>
+<tr>
+<td>Query</td>
+<td>enable.unsafe.columnpage</td>
+<td>TRUE</td>
+<td>Keep the column pages in offheap memory so that the memory overhead due to java object is less and also reduces GC pressure.</td>
+</tr>
+<tr>
+<td>Query</td>
+<td>carbon.unsafe.working.memory.in.mb</td>
+<td>10240</td>
+<td>Amount of memory to use for offheap operations.Can increase this memory based on the data size</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="results-achieved-1" class="anchor" href="#results-achieved-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Results Achieved</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Results</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>Query (Time Period spanning 1 segment)</td>
+<td>&lt; 10 Sec</td>
+</tr>
+<tr>
+<td>Data Loading Speed</td>
+<td>45 MB/s Per Node</td>
+</tr>
+</tbody>
+</table>
+<h2>
+<a id="olapbi-queries-in-the-webinternet-scenario" class="anchor" href="#olapbi-queries-in-the-webinternet-scenario" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>OLAP/BI Queries in the web/Internet scenario</h2>
+<h3>
+<a id="scenario-2" class="anchor" href="#scenario-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Scenario</h3>
+<p>An Internet company wants to analyze the average download speed, kind of handsets used in a particular region/area,kind of Apps being used, what kind of videos are trending in a particular region to enable them to identify the appropriate resolution size of videos to speed up transfer, and perform many more analysis to serve th customers better.</p>
+<h3>
+<a id="challenges-2" class="anchor" href="#challenges-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Challenges</h3>
+<p>Since data is being queried by a BI tool, all the queries contain group by, which means CarbonData need to return more records as limit cannot be pushed down to carbondata layer.</p>
+<p>Results have to be returned faster as the BI tool would not respond till the data is fetched, causing bad user experience.</p>
+<p>Data might be loaded less frequently(once or twice in a day), but raw data size is huge, which causes the group by queries to run slower.</p>
+<p>Concurrent queries can be more due to the BI dashboard</p>
+<h3>
+<a id="goal" class="anchor" href="#goal" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Goal</h3>
+<ol>
+<li>Aggregation queries are faster</li>
+<li>Concurrency is high(Number of concurrent queries supported)</li>
+</ol>
+<h3>
+<a id="solution-2" class="anchor" href="#solution-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Solution</h3>
+<ul>
+<li>Use table block size as 128MB so that pruning is more effective</li>
+<li>Use global sort mode so that the data to be fetched are grouped together</li>
+<li>Create pre-aggregate tables for non timestamp based group by queries</li>
+<li>For queries containing group by date, create timeseries based Datamap(pre-aggregate) tables so that the data is rolled up during creation and fetch is faster</li>
+<li>Reduce the Spark shuffle partitions.(In our configuration on 14 node cluster, it was reduced to 35 from default of 200)</li>
+<li>Enable global dictionary for columns which have less cardinalities.Aggregation can be done on encoded data, there by improving the performance</li>
+<li>For columns whose cardinality is high,enable the local dictionary so that store size is less and can take dictionary benefit for scan</li>
+</ul>
+<h2>
+<a id="handling-near-realtime-data-ingestion-scenario" class="anchor" href="#handling-near-realtime-data-ingestion-scenario" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Handling near realtime data ingestion scenario</h2>
+<h3>
+<a id="scenario-3" class="anchor" href="#scenario-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Scenario</h3>
+<p>Need to support storing of continously arriving data and make it available immediately for query.</p>
+<h3>
+<a id="challenges-3" class="anchor" href="#challenges-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Challenges</h3>
+<p>When the data ingestion is near real time and the data needs to be available for query immediately, usual scenario is to do data loading in micro batches.But this causes the problem of generating many small files.This poses two problems:</p>
+<ol>
+<li>Small file handling in HDFS is inefficient</li>
+<li>CarbonData will suffer in query performance as all the small files will have to be queried when filter is on non time column</li>
+</ol>
+<p>CarbonData will suffer in query performance as all the small files will have to be queried when filter is on non time column.</p>
+<p>Since data is continouly arriving, allocating resources for compaction might not be feasible.</p>
+<h3>
+<a id="goal-1" class="anchor" href="#goal-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Goal</h3>
+<ol>
+<li>Data is available in near real time for query as it arrives</li>
+<li>CarbonData doesnt suffer from small files problem</li>
+</ol>
+<h3>
+<a id="solution-3" class="anchor" href="#solution-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Solution</h3>
+<ul>
+<li>Use Streaming tables support of CarbonData</li>
+<li>Configure the carbon.streaming.segment.max.size property to higher value(default is 1GB) if a bit slower query performance is not a concern</li>
+<li>Configure carbon.streaming.auto.handoff.enabled to true so that after the  carbon.streaming.segment.max.size is reached, the segment is converted into format optimized for query</li>
+<li>Disable auto compaction.Manually trigger the minor compaction with default 4,3 when the cluster is not busy</li>
+<li>Manually trigger Major compaction based on the size of segments and the frequency with which the segments are being created</li>
+<li>Enable local dictionary</li>
+</ul>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__uses').addClass('selected'); });
+</script></div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/useful-tips-on-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/useful-tips-on-carbondata.html b/src/main/webapp/useful-tips-on-carbondata.html
deleted file mode 100644
index eba1b0a..0000000
--- a/src/main/webapp/useful-tips-on-carbondata.html
+++ /dev/null
@@ -1,542 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
-                                   target="_blank">Apache CarbonData 1.4.1</a></li>
-							<li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
-                                   target="_blank">Apache CarbonData 1.4.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
-                                   target="_blank">Apache CarbonData 1.3.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
-                                   target="_blank">Apache CarbonData 1.3.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="documentation.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search" target="_blank">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input"  placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="verticalnavbar">
-                <nav class="b-sticky-nav">
-                    <div class="nav-scroller">
-                        <div class="nav__inner">
-                            <a class="b-nav__home nav__item" href="/">home</a>
-                            <a class="b-nav__intro nav__item" href="/intro">introduction</a>
-                            <a class="b-nav__quickstart nav__item" href="/quickstart">quickstart</a>
-                            <a class="b-nav__uses nav__item" href="/uses">use cases</a>
-
-                            <div class="nav__item nav__item__with__subs">
-                                <a class="b-nav__docs nav__item nav__sub__anchor" href="/documentation">documentation</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#gettingStarted">getting started</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#api">APIs</a>
-                                <a class="b-nav__streams nav__item nav__sub__item" href="/documentation/streams">kafka streams</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#connect">kafka connect</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#configuration">configuration</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#design">design</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#implementation">implementation</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#operations">operations</a>
-                                <a class="nav__item nav__sub__item" href="/documentation#security">security</a>
-                            </div>
-
-                            <a class="b-nav__performance nav__item" href="/performance">performance</a>
-                            <a class="b-nav__poweredby nav__item" href="/powered-by">powered by</a>
-                            <a class="b-nav__project nav__item" href="/project">project info</a>
-                            <a class="b-nav__ecosystem nav__item" href="https://cwiki.apache.org/confluence/display/KAFKA/Ecosystem" target="_blank">ecosystem</a>
-                            <a class="b-nav__clients nav__item" href="https://cwiki.apache.org/confluence/display/KAFKA/Clients" target="_blank">clients</a>
-                            <a class="b-nav__events nav__item" href="/events">events</a>
-                            <a class="b-nav__contact nav__item" href="/contact">contact us</a>
-
-                            <div class="nav__item nav__item__with__subs">
-                                <a class="b-nav__apache nav__item nav__sub__anchor b-nav__sub__anchor" href="#">apache</a>
-                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/" target="_blank">foundation</a>
-                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/licenses/" target="_blank">license</a>
-                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/foundation/sponsorship.html" target="_blank">sponsorship</a>
-                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/foundation/thanks.html" target="_blank">thanks</a>
-                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/security/" target="_blank">security</a>
-                            </div>
-
-                            <a class="btn" href="/downloads">download</a>
-                            <div class="social-links">
-                                <a class="twitter" href="https://twitter.com/apachekafka" target="_blank">@apachekafka</a>
-                            </div>
-                        </div>
-                    </div>
-                    <div class="navindicator">
-                        <div class="b-nav__home navindicator__item"></div>
-                        <div class="b-nav__intro navindicator__item"></div>
-                        <div class="b-nav__quickstart navindicator__item"></div>
-                        <div class="b-nav__uses navindicator__item"></div>
-                        <div class="b-nav__docs navindicator__item"></div>
-                        <div class="b-nav__performance navindicator__item"></div>
-                        <div class="b-nav__poweredby navindicator__item"></div>
-                        <div class="b-nav__project navindicator__item"></div>
-                        <div class="b-nav__ecosystem navindicator__item"></div>
-                        <div class="b-nav__clients navindicator__item"></div>
-                        <div class="b-nav__events navindicator__item"></div>
-                        <div class="b-nav__contact navindicator__item"></div>
-                    </div>
-                </nav>
-            </div>
-            <div class="mdcontent">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="row">
-                                <div class="col-sm-12  col-md-12">
-                                    <div>
-<h1>
-<a id="useful-tips" class="anchor" href="#useful-tips" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Useful Tips</h1>
-<p>This tutorial guides you to create CarbonData Tables and optimize performance.
-The following sections will elaborate on the below topics :</p>
-<ul>
-<li><a href="#suggestions-to-create-carbondata-table">Suggestions to create CarbonData Table</a></li>
-<li><a href="#configuration-for-optimizing-data-loading-performance-for-massive-data">Configuration for Optimizing Data Loading performance for Massive Data</a></li>
-<li><a href="#configurations-for-optimizing-carbondata-performance">Optimizing Mass Data Loading</a></li>
-</ul>
-<h2>
-<a id="suggestions-to-create-carbondata-table" class="anchor" href="#suggestions-to-create-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Suggestions to Create CarbonData Table</h2>
-<p>For example, the results of the analysis for table creation with dimensions ranging from 10 thousand to 10 billion rows and 100 to 300 columns have been summarized below.
-The following table describes some of the columns from the table used.</p>
-<ul>
-<li><strong>Table Column Description</strong></li>
-</ul>
-<table>
-<thead>
-<tr>
-<th>Column Name</th>
-<th>Data Type</th>
-<th>Cardinality</th>
-<th>Attribution</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>msisdn</td>
-<td>String</td>
-<td>30 million</td>
-<td>Dimension</td>
-</tr>
-<tr>
-<td>BEGIN_TIME</td>
-<td>BigInt</td>
-<td>10 Thousand</td>
-<td>Dimension</td>
-</tr>
-<tr>
-<td>HOST</td>
-<td>String</td>
-<td>1 million</td>
-<td>Dimension</td>
-</tr>
-<tr>
-<td>Dime_1</td>
-<td>String</td>
-<td>1 Thousand</td>
-<td>Dimension</td>
-</tr>
-<tr>
-<td>counter_1</td>
-<td>Decimal</td>
-<td>NA</td>
-<td>Measure</td>
-</tr>
-<tr>
-<td>counter_2</td>
-<td>Numeric(20,0)</td>
-<td>NA</td>
-<td>Measure</td>
-</tr>
-<tr>
-<td>...</td>
-<td>...</td>
-<td>NA</td>
-<td>Measure</td>
-</tr>
-<tr>
-<td>counter_100</td>
-<td>Decimal</td>
-<td>NA</td>
-<td>Measure</td>
-</tr>
-</tbody>
-</table>
-<ul>
-<li><strong>Put the frequently-used column filter in the beginning</strong></li>
-</ul>
-<p>For example, MSISDN filter is used in most of the query then we must put the MSISDN in the first column.
-The create table command can be modified as suggested below :</p>
-<pre><code>create table carbondata_table(
-  msisdn String,
-  BEGIN_TIME bigint,
-  HOST String,
-  Dime_1 String,
-  counter_1, Decimal
-  ...
-  
-  )STORED BY 'carbondata'
-  TBLPROPERTIES ('SORT_COLUMNS'='msisdn, Dime_1')
-</code></pre>
-<p>Now the query with MSISDN in the filter will be more efficient.</p>
-<ul>
-<li><strong>Put the frequently-used columns in the order of low to high cardinality</strong></li>
-</ul>
-<p>If the table in the specified query has multiple columns which are frequently used to filter the results, it is suggested to put
-the columns in the order of cardinality low to high. This ordering of frequently used columns improves the compression ratio and
-enhances the performance of queries with filter on these columns.</p>
-<p>For example, if MSISDN, HOST and Dime_1 are frequently-used columns, then the column order of table is suggested as
-Dime_1&gt;HOST&gt;MSISDN, because Dime_1 has the lowest cardinality.
-The create table command can be modified as suggested below :</p>
-<pre><code>create table carbondata_table(
-    msisdn String,
-    BEGIN_TIME bigint,
-    HOST String,
-    Dime_1 String,
-    counter_1, Decimal
-    ...
-    
-    )STORED BY 'carbondata'
-    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
-</code></pre>
-<ul>
-<li><strong>For measure type columns with non high accuracy, replace Numeric(20,0) data type with Double data type</strong></li>
-</ul>
-<p>For columns of measure type, not requiring high accuracy, it is suggested to replace Numeric data type with Double to enhance query performance.
-The create table command can be modified as below :</p>
-<pre><code>  create table carbondata_table(
-    Dime_1 String,
-    BEGIN_TIME bigint,
-    END_TIME bigint,
-    HOST String,
-    MSISDN String,
-    counter_1 decimal,
-    counter_2 double,
-    ...
-    )STORED BY 'carbondata'
-    TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
-</code></pre>
-<p>The result of performance analysis of test-case shows reduction in query execution time from 15 to 3 seconds, thereby improving performance by nearly 5 times.</p>
-<ul>
-<li><strong>Columns of incremental character should be re-arranged at the end of dimensions</strong></li>
-</ul>
-<p>Consider the following scenario where data is loaded each day and the begin_time is incremental for each load, it is suggested to put begin_time at the end of dimensions.
-Incremental values are efficient in using min/max index. The create table command can be modified as below :</p>
-<pre><code>create table carbondata_table(
-  Dime_1 String,
-  HOST String,
-  MSISDN String,
-  counter_1 double,
-  counter_2 double,
-  BEGIN_TIME bigint,
-  END_TIME bigint,
-  ...
-  counter_100 double
-  )STORED BY 'carbondata'
-  TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>BloomFilter can be created to enhance performance for queries with precise equal/in conditions. You can find more information about it in BloomFilter datamap <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/bloomfilter-datamap-guide.html" target=_blank>document</a>.</li>
-</ul>
-<h2>
-<a id="configuration-for-optimizing-data-loading-performance-for-massive-data" class="anchor" href="#configuration-for-optimizing-data-loading-performance-for-massive-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configuration for Optimizing Data Loading performance for Massive Data</h2>
-<p>CarbonData supports large data load, in this process sorting data while loading consumes a lot of memory and disk IO and
-this can result sometimes in "Out Of Memory" exception.
-If you do not have much memory to use, then you may prefer to slow the speed of data loading instead of data load failure.
-You can configure CarbonData by tuning following properties in carbon.properties file to get a better performance.</p>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Default Value</th>
-<th>Description/Tuning</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>carbon.number.of.cores.while.loading</td>
-<td>Default: 2.This value should be &gt;= 2</td>
-<td>Specifies the number of cores used for data processing during data loading in CarbonData.</td>
-</tr>
-<tr>
-<td>carbon.sort.size</td>
-<td>Default: 100000. The value should be &gt;= 100.</td>
-<td>Threshold to write local file in sort step when loading data</td>
-</tr>
-<tr>
-<td>carbon.sort.file.write.buffer.size</td>
-<td>Default:  50000.</td>
-<td>DataOutputStream buffer.</td>
-</tr>
-<tr>
-<td>carbon.number.of.cores.block.sort</td>
-<td>Default: 7</td>
-<td>If you have huge memory and CPUs, increase it as you will</td>
-</tr>
-<tr>
-<td>carbon.merge.sort.reader.thread</td>
-<td>Default: 3</td>
-<td>Specifies the number of cores used for temp file merging during data loading in CarbonData.</td>
-</tr>
-<tr>
-<td>carbon.merge.sort.prefetch</td>
-<td>Default: true</td>
-<td>You may want set this value to false if you have not enough memory</td>
-</tr>
-</tbody>
-</table>
-<p>For example, if there are 10 million records, and i have only 16 cores, 64GB memory, will be loaded to CarbonData table.
-Using the default configuration  always fail in sort step. Modify carbon.properties as suggested below:</p>
-<pre><code>carbon.number.of.cores.block.sort=1
-carbon.merge.sort.reader.thread=1
-carbon.sort.size=5000
-carbon.sort.file.write.buffer.size=5000
-carbon.merge.sort.prefetch=false
-</code></pre>
-<h2>
-<a id="configurations-for-optimizing-carbondata-performance" class="anchor" href="#configurations-for-optimizing-carbondata-performance" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configurations for Optimizing CarbonData Performance</h2>
-<p>Recently we did some performance POC on CarbonData for Finance and telecommunication Field. It involved detailed queries and aggregation
-scenarios. After the completion of POC, some of the configurations impacting the performance have been identified and tabulated below :</p>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Location</th>
-<th>Used For</th>
-<th>Description</th>
-<th>Tuning</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>carbon.sort.intermediate.files.limit</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>During the loading of data, local temp is used to sort the data. This number specifies the minimum number of intermediate files after which the  merge sort has to be initiated.</td>
-<td>Increasing the parameter to a higher value will improve the load performance. For example, when we increase the value from 20 to 100, it increases the data load performance from 35MB/S to more than 50MB/S. Higher values of this parameter consumes  more memory during the load.</td>
-</tr>
-<tr>
-<td>carbon.number.of.cores.while.loading</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>Specifies the number of cores used for data processing during data loading in CarbonData.</td>
-<td>If you have more number of CPUs, then you can increase the number of CPUs, which will increase the performance. For example if we increase the value from 2 to 4 then the CSV reading performance can increase about 1 times</td>
-</tr>
-<tr>
-<td>carbon.compaction.level.threshold</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading and Querying</td>
-<td>For minor compaction, specifies the number of segments to be merged in stage 1 and number of compacted segments to be merged in stage 2.</td>
-<td>Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance. Configuring this parameter will merge the small segment to one big segment which will sort the data and improve the performance. For Example in one telecommunication scenario, the performance improves about 2 times after minor compaction.</td>
-</tr>
-<tr>
-<td>spark.sql.shuffle.partitions</td>
-<td>spark/conf/spark-defaults.conf</td>
-<td>Querying</td>
-<td>The number of task started when spark shuffle.</td>
-<td>The value can be 1 to 2 times as much as the executor cores. In an aggregation scenario, reducing the number from 200 to 32 reduced the query time from 17 to 9 seconds.</td>
-</tr>
-<tr>
-<td>spark.executor.instances/spark.executor.cores/spark.executor.memory</td>
-<td>spark/conf/spark-defaults.conf</td>
-<td>Querying</td>
-<td>The number of executors, CPU cores, and memory used for CarbonData query.</td>
-<td>In the bank scenario, we provide the 4 CPUs cores and 15 GB for each executor which can get good performance. This 2 value does not mean more the better. It needs to be configured properly in case of limited resources. For example, In the bank scenario, it has enough CPU 32 cores each node but less memory 64 GB each node. So we cannot give more CPU but less memory. For example, when 4 cores and 12GB for each executor. It sometimes happens GC during the query which impact the query performance very much from the 3 second to more than 15 seconds. In this scenario need to increase the memory or decrease the CPU cores.</td>
-</tr>
-<tr>
-<td>carbon.detail.batch.size</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>The buffer size to store records, returned from the block scan.</td>
-<td>In limit scenario this parameter is very important. For example your query limit is 1000. But if we set this value to 3000 that means we get 3000 records from scan but spark will only take 1000 rows. So the 2000 remaining are useless. In one Finance test case after we set it to 100, in the limit 1000 scenario the performance increase about 2 times in comparison to if we set this value to 12000.</td>
-</tr>
-<tr>
-<td>carbon.use.local.dir</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>Whether use YARN local directories for multi-table load disk load balance</td>
-<td>If this is set it to true CarbonData will use YARN local directories for multi-table load disk load balance, that will improve the data load performance.</td>
-</tr>
-<tr>
-<td>carbon.use.multiple.temp.dir</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>Whether to use multiple YARN local directories during table data loading for disk load balance</td>
-<td>After enabling 'carbon.use.local.dir', if this is set to true, CarbonData will use all YARN local directories during data load for disk load balance, that will improve the data load performance. Please enable this property when you encounter disk hotspot problem during data loading.</td>
-</tr>
-<tr>
-<td>carbon.sort.temp.compressor</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>Specify the name of compressor to compress the intermediate sort temporary files during sort procedure in data loading.</td>
-<td>The optional values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files. This parameter will be useful if you encounter disk bottleneck.</td>
-</tr>
-<tr>
-<td>carbon.load.skewedDataOptimization.enabled</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>Whether to enable size based block allocation strategy for data loading.</td>
-<td>When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data -- It's useful if the size of your input data files varies widely, say 1MB~1GB.</td>
-</tr>
-<tr>
-<td>carbon.load.min.size.enabled</td>
-<td>spark/carbonlib/carbon.properties</td>
-<td>Data loading</td>
-<td>Whether to enable node minumun input data size allocation strategy for data loading.</td>
-<td>When loading, carbondata will use node minumun input data size allocation strategy for task distribution. It will make sure the node load the minimum amount of data -- It's useful if the size of your input data files very small, say 1MB~256MB,Avoid generating a large number of small files.</td>
-</tr>
-</tbody>
-</table>
-<p>Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.</p>
-</div>
-</div>
-</div>
-</div>
-<div class="doc-footer">
-    <a href="#top" class="scroll-top">Top</a>
-</div>
-</div>
-</section>
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/videogallery.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/videogallery.html b/src/main/webapp/videogallery.html
index e494174..c282364 100644
--- a/src/main/webapp/videogallery.html
+++ b/src/main/webapp/videogallery.html
@@ -246,4 +246,4 @@
 
 
 </body>
-</html>
+</html>
\ No newline at end of file


[04/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/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
deleted file mode 100644
index 0c54535..0000000
--- a/src/site/markdown/data-management-on-carbondata.md
+++ /dev/null
@@ -1,1394 +0,0 @@
-<!--
-    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)
-* [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 AS carbondata
-  [TBLPROPERTIES (property_name=property_value, ...)]
-  [LOCATION 'path']
-  ```
-  **NOTE:** CarbonData also supports "STORED AS carbondata" and "USING carbondata". Find example code at [CarbonSessionExample](https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala) in the CarbonData repo.
-### 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')
-	 ```
-	 NOTE: Dictionary Include/Exclude for complex child columns is not supported.
-	 
-   - **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'='')
-     ```
-     NOTE: Sort_Columns for Complex datatype columns is not supported.
-
-   - **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. 
-	 
-	### 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 ('SORT_COLUMNS'='productName,storeCity',
-                   'SORT_SCOPE'='NO_SORT')
-   ```
-   
-   **NOTE:** CarbonData also supports "using carbondata". Find example code at [SparkSessionExample](https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala) in the CarbonData repo.
- 
-   - **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 as carbon.major.compaction.size, size in MB.
-     * AUTO_LOAD_MERGE: same meaning as carbon.enable.auto.load.merge.
-     * COMPACTION_LEVEL_THRESHOLD: same meaning as carbon.compaction.level.threshold.
-     * COMPACTION_PRESERVE_SEGMENTS: same meaning as carbon.numberof.preserve.segments.
-     * ALLOWED_COMPACTION_DAYS: same meaning as 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')
-     ```
-
-   - **Local Dictionary Configuration**
-   
-   Columns for which dictionary is not generated needs more storage space and in turn more IO. Also since more data will have to be read during query, query performance also would suffer.Generating dictionary per blocklet for such columns would help in saving storage space and assist in improving query performance as carbondata is optimized for handling dictionary encoded columns more effectively.Generating dictionary internally per blocklet is termed as local dictionary. Please refer to [File structure of Carbondata](../file-structure-of-carbondata.md) for understanding about the file structure of carbondata and meaning of terms like blocklet.
-   
-   Local Dictionary helps in:
-   1. Getting more compression.
-   2. Filter queries and full scan queries will be faster as filter will be done on encoded data.
-   3. Reducing the store size and memory footprint as only unique values will be stored as part of local dictionary and corresponding data will be stored as encoded data.
-   4. Getting higher IO throughput.
- 
-   **NOTE:** 
-   
-   * Following Data Types are Supported for Local Dictionary:
-      * STRING
-      * VARCHAR
-      * CHAR
-
-   * Following Data Types are not Supported for Local Dictionary: 
-      * SMALLINT
-      * INTEGER
-      * BIGINT
-      * DOUBLE
-      * DECIMAL
-      * TIMESTAMP
-      * DATE
-      * BOOLEAN
-   
-   * In case of multi-level complex dataType columns, primitive string/varchar/char columns are considered for local dictionary generation.
-   
-   Local dictionary will have to be enabled explicitly during create table or by enabling the system property 'carbon.local.dictionary.enable'. By default, Local Dictionary will be disabled for the carbondata table.
-    
-   Local Dictionary can be configured using the following properties during create table command: 
-          
-   | Properties | Default value | Description |
-   | ---------- | ------------- | ----------- |
-   | LOCAL_DICTIONARY_ENABLE | false | Whether to enable local dictionary generation. **NOTE:** If this property is defined, it will override the value configured at system level by 'carbon.local.dictionary.enable' |
-   | LOCAL_DICTIONARY_THRESHOLD | 10000 | The maximum cardinality of a column upto which carbondata can try to generate local dictionary (maximum - 100000) |
-   | LOCAL_DICTIONARY_INCLUDE | string/varchar/char columns| Columns for which Local Dictionary has to be generated.**NOTE:** Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.|
-   | LOCAL_DICTIONARY_EXCLUDE | none | Columns for which Local Dictionary need not be generated. |
-        
-   **Fallback behavior:** 
-   
-   * When the cardinality of a column exceeds the threshold, it triggers a fallback and the generated dictionary will be reverted and data loading will be continued without dictionary encoding.
-   
-   **NOTE:** When fallback is triggered, the data loading performance will decrease as encoded data will be discarded and the actual data is written to the temporary sort files.
-   
-   **Points to be noted:**
-      
-   1. Reduce Block size:
-   
-      Number of Blocks generated is less in case of Local Dictionary as compression ratio is high. This may reduce the number of tasks launched during query, resulting in degradation of query performance if the pruned blocks are less compared to the number of parallel tasks which can be run. So it is recommended to configure smaller block size which in turn generates more number of blocks.
-            
-   2. All the page-level data for a blocklet needs to be maintained in memory until all the pages encoded for local dictionary is processed in order to handle fallback. Hence the memory required for local dictionary based table is more and this memory increase is proportional to number of columns. 
-       
-### Example:
- 
-   ```
-   CREATE TABLE carbontable(
-             
-               column1 string,
-             
-               column2 string,
-             
-               column3 LONG )
-             
-     STORED BY 'carbondata'
-     TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='true','LOCAL_DICTIONARY_THRESHOLD'='1000',
-     'LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
-   ```
-
-   **NOTE:** 
-   
-   * We recommend to use Local Dictionary when cardinality is high but is distributed across multiple loads
-   * On a large cluster, decoding data can become a bottleneck for global dictionary as there will be many remote reads. In this scenario, it is better to use Local Dictionary.
-   * When cardinality is less, but loads are repetitive, it is better to use global dictionary as local dictionary generates multiple dictionary files at blocklet level increasing redundancy.
-
-   - **Caching Min/Max Value for Required Columns**
-     By default, CarbonData caches min and max values of all the columns in schema.  As the load increases, the memory required to hold the min and max values increases considerably. This feature enables you to configure min and max values only for the required columns, resulting in optimized memory usage. 
-	 
-	 Following are the valid values for COLUMN_META_CACHE:
-	 * If you want no column min/max values to be cached in the driver.
-	 
-	 ```
-	 COLUMN_META_CACHE=’’
-	 ```
-	 
-	 * If you want only col1 min/max values to be cached in the driver.
-	 
-	 ```
-	 COLUMN_META_CACHE=’col1’
-	 ```
-	 
-	 * If you want min/max values to be cached in driver for all the specified columns.
-	 
-	 ```
-	 COLUMN_META_CACHE=’col1,col2,col3,…’
-	 ```
-	 
-	 Columns to be cached can be specified either while creating table or after creation of the table.
-	 During create table operation; specify the columns to be cached in table properties.
-	 
-	 Syntax:
-	 
-	 ```
-	 CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,…) STORED BY ‘carbondata’ TBLPROPERTIES (‘COLUMN_META_CACHE’=’col1,col2,…’)
-	 ```
-	 
-	 Example:
-	 
-	 ```
-	 CREATE TABLE employee (name String, city String, id int) STORED BY ‘carbondata’ TBLPROPERTIES (‘COLUMN_META_CACHE’=’name’)
-	 ```
-	 
-	 After creation of table or on already created tables use the alter table command to configure the columns to be cached.
-	 
-	 Syntax:
-	 
-	 ```
-	 ALTER TABLE [dbName].tableName SET TBLPROPERTIES (‘COLUMN_META_CACHE’=’col1,col2,…’)
-	 ```
-	 
-	 Example:
-	 
-	 ```
-	 ALTER TABLE employee SET TBLPROPERTIES (‘COLUMN_META_CACHE’=’city’)
-	 ```
-	 
-   - **Caching at Block or Blocklet Level**
-
-     This feature allows you to maintain the cache at Block level, resulting in optimized usage of the memory. The memory consumption is high if the Blocklet level caching is maintained as a Block can have multiple Blocklet.
-	 
-	 Following are the valid values for CACHE_LEVEL:
-
-	 *Configuration for caching in driver at Block level (default value).*
-	 
-	 ```
-	 CACHE_LEVEL= ‘BLOCK’
-	 ```
-	 
-	 *Configuration for caching in driver at Blocklet level.*
-	 
-	 ```
-	 CACHE_LEVEL= ‘BLOCKLET’
-	 ```
-	 
-	 Cache level can be specified either while creating table or after creation of the table.
-	 During create table operation specify the cache level in table properties.
-	 
-	 Syntax:
-	 
-	 ```
-	 CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,…) STORED BY ‘carbondata’ TBLPROPERTIES (‘CACHE_LEVEL’=’Blocklet’)
-	 ```
-	 
-	 Example:
-	 
-	 ```
-	 CREATE TABLE employee (name String, city String, id int) STORED BY ‘carbondata’ TBLPROPERTIES (‘CACHE_LEVEL’=’Blocklet’)
-	 ```
-	 
-	 After creation of table or on already created tables use the alter table command to configure the cache level.
-	 
-	 Syntax:
-	 
-	 ```
-	 ALTER TABLE [dbName].tableName SET TBLPROPERTIES (‘CACHE_LEVEL’=’Blocklet’)
-	 ```
-	 
-	 Example:
-	 
-	 ```
-	 ALTER TABLE employee SET TBLPROPERTIES (‘CACHE_LEVEL’=’Blocklet’)
-	 ```
-
-    - **Support Flat folder same as Hive/Parquet**
-
-	  This feature allows all carbondata and index files to keep directy under tablepath. Currently all carbondata/carbonindex files written under tablepath/Fact/Part0/Segment_NUM folder and it is not same as hive/parquet folder structure. This feature makes all files written will be directly under tablepath, it does not maintain any segment folder structure.This is useful for interoperability between the execution engines and plugin with other execution engines like hive or presto becomes easier.
-
-	  Following table property enables this feature and default value is false.
-	  ```
-	   'flat_folder'='true'
-	  ```
-	  Example:
-	  ```
-	  CREATE TABLE employee (name String, city String, id int) STORED BY ‘carbondata’ TBLPROPERTIES ('flat_folder'='true')
-	  ```
-
-    - **String longer than 32000 characters**
-
-     In common scenarios, the length of string is less than 32000,
-     so carbondata stores the length of content using Short to reduce memory and space consumption.
-     To support string longer than 32000 characters, carbondata introduces a table property called `LONG_STRING_COLUMNS`.
-     For these columns, carbondata internally stores the length of content using Integer.
-
-     You can specify the columns as 'long string column' using below tblProperties:
-
-     ```
-     // specify col1, col2 as long string columns
-     TBLPROPERTIES ('LONG_STRING_COLUMNS'='col1,col2')
-     ```
-
-     Besides, you can also use this property through DataFrame by
-     ```
-     df.format("carbondata")
-       .option("tableName", "carbonTable")
-       .option("long_string_columns", "col1, col2")
-       .save()
-     ```
-
-     If you are using Carbon-SDK, you can specify the datatype of long string column as `varchar`.
-     You can refer to SDKwriterTestCase for example.
-
-     **NOTE:** The LONG_STRING_COLUMNS can only be string/char/varchar columns and cannot be dictionary_include/sort_columns/complex columns.
-
-## CREATE TABLE AS SELECT
-  This function allows user 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
-  ```
-  carbon.sql("CREATE TABLE source_table(
-                             id INT,
-                             name STRING,
-                             city STRING,
-                             age INT)
-              STORED AS parquet")
-  carbon.sql("INSERT INTO source_table SELECT 1,'bob','shenzhen',27")
-  carbon.sql("INSERT INTO source_table SELECT 2,'david','shenzhen',31")
-  
-  carbon.sql("CREATE TABLE target_table
-              STORED BY 'carbondata'
-              AS SELECT city,avg(age) FROM source_table GROUP BY city")
-              
-  carbon.sql("SELECT * FROM target_table").show
-    // results:
-    //    +--------+--------+
-    //    |    city|avg(age)|
-    //    +--------+--------+
-    //    |shenzhen|    29.0|
-    //    +--------+--------+
-
-  ```
-
-## CREATE EXTERNAL TABLE
-  This function allows user to create external table by specifying location.
-  ```
-  CREATE EXTERNAL TABLE [IF NOT EXISTS] [db_name.]table_name 
-  STORED BY 'carbondata' LOCATION ‘$FilesPath’
-  ```
-  
-### Create external table on managed table data location.
-  Managed table data location provided will have both FACT and Metadata folder. 
-  This data can be generated by creating a normal carbon table and use this path as $FilesPath in the above syntax.
-  
-  **Example:**
-  ```
-  sql("CREATE TABLE origin(key INT, value STRING) STORED BY 'carbondata'")
-  sql("INSERT INTO origin select 100,'spark'")
-  sql("INSERT INTO origin select 200,'hive'")
-  // creates a table in $storeLocation/origin
-  
-  sql(s"""
-  |CREATE EXTERNAL TABLE source
-  |STORED BY 'carbondata'
-  |LOCATION '$storeLocation/origin'
-  """.stripMargin)
-  checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin"))
-  ```
-  
-### Create external table on Non-Transactional table data location.
-  Non-Transactional table data location will have only carbondata and carbonindex files, there will not be a metadata folder (table status and schema).
-  Our SDK module currently support writing data in this format.
-  
-  **Example:**
-  ```
-  sql(
-  s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbondata' LOCATION
-  |'$writerPath' """.stripMargin)
-  ```
-  
-  Here writer path will have carbondata and index files.
-  This can be SDK output. Refer [SDK Writer Guide](https://github.com/apache/carbondata/blob/master/docs/sdk-writer-guide.md). 
-  
-  **Note:**
-  1. Dropping of the external table should not delete the files present in the location.
-  2. When external table is created on non-transactional table data, 
-  external table will be registered with the schema of carbondata files.
-  If multiple files with different schema is present, exception will be thrown.
-  So, If table registered with one schema and files are of different schema, 
-  suggest to drop the external table and create again to register table with new schema.  
-
-
-## 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”;
-  ```
-
-## 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 TABLE
-
-  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 carbonTable
-     OR
-     ALTER TABLE test_db.carbon RENAME TO test_db.carbonTable
-     ```
-
-   - **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')
-     ```
-      NOTE: Add Complex datatype columns is not supported.
-
-Users can specify which columns to include and exclude for local dictionary generation after adding new columns. These will be appended with the already existing local dictionary include and exclude columns of main table respectively.
-  ```
-     ALTER TABLE carbon ADD COLUMNS (a1 STRING, b1 STRING) TBLPROPERTIES('LOCAL_DICTIONARY_INCLUDE'='a1','LOCAL_DICTIONARY_EXCLUDE'='b1')
-  ```
-
-   - **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)
-     ```
-     NOTE: Drop Complex child column is not supported.
-
-   - **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)
-     ```
-- **MERGE INDEX**
-   
-     This command is used to merge all the CarbonData index files (.carbonindex) inside a segment to a single CarbonData index merge file (.carbonindexmerge). This enhances the first query performance.
-     ```
-      ALTER TABLE [db_name.]table_name COMPACT 'SEGMENT_INDEX'
-      ```
-      
-      Examples:
-      ```
-      ALTER TABLE test_db.carbon COMPACT 'SEGMENT_INDEX'
-      ```
-      **NOTE:**
-      * Merge index is not supported on streaming table.
-      
-- **SET and UNSET for Local Dictionary Properties**
-
-   When set command is used, all the newly set properties will override the corresponding old properties if exists.
-  
-   Example to SET Local Dictionary Properties:
-    ```
-   ALTER TABLE tablename SET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='false','LOCAL_DICTIONARY_THRESHOLD'='1000','LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
-    ```
-   When Local Dictionary properties are unset, corresponding default values will be used for these properties.
-      
-   Example to UNSET Local Dictionary Properties:
-    ```
-   ALTER TABLE tablename UNSET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE','LOCAL_DICTIONARY_THRESHOLD','LOCAL_DICTIONARY_INCLUDE','LOCAL_DICTIONARY_EXCLUDE')
-    ```
-    
-   **NOTE:** For old tables, by default, local dictionary is disabled. If user wants local dictionary for these tables, user can enable/disable local dictionary for new data at their discretion. 
-   This can be achieved by using the alter table set command.
-
-### 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 used HIVE meta store to store schema, refresh will not work as schema file does not exist in file system.
-
-### Table and Column Comment
-
-  You can provide more information on table by using table comment. Similarly you can provide more information about a particular column using column comment. 
-  You can see the column comment of an existing table using describe formatted command.
-  
-  ```
-  CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type [COMMENT col_comment], ...)]
-    [COMMENT table_comment]
-  STORED BY 'carbondata'
-  [TBLPROPERTIES (property_name=property_value, ...)]
-  ```
-  
-  Example:
-  ```
-  CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
-                                productNumber Int COMMENT 'unique serial number for product')
-  COMMENT “This is table comment”
-   STORED BY 'carbondata'
-   TBLPROPERTIES ('DICTIONARY_INCLUDE'='productNumber')
-  ```
-  You can also SET and UNSET table comment using ALTER command.
-
-  Example to SET table comment:
-  
-  ```
-  ALTER TABLE carbon SET TBLPROPERTIES ('comment'='this table comment is modified');
-  ```
-  
-  Example to UNSET table comment:
-  
-  ```
-  ALTER TABLE carbon UNSET TBLPROPERTIES ('comment');
-  ```
-
-## 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 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'='#')
-    ```
-
-  - **HEADER:** When you load the CSV file without the file header and the file header is the same with the table schema, then add 'HEADER'='false' to load data SQL as user need not provide the file header. By default the value is 'true'.
-  false: CSV file is without file header.
-  true: CSV file is with file header.
-  
-    ```
-    OPTIONS('HEADER'='false') 
-    ```
-
-	**NOTE:** If the HEADER option exist and is set to 'true', then the FILEHEADER option is not required.
-	
-  - **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 in CSV files.
-
-    ```
-    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).
-
-  - **SORT COLUMN BOUNDS:** Range bounds for sort columns.
-
-    Suppose the table is created with 'SORT_COLUMNS'='name,id' and the range for name is aaa~zzz, the value range for id is 0~1000. Then during data loading, we can specify the following option to enhance data loading performance.
-    ```
-    OPTIONS('SORT_COLUMN_BOUNDS'='f,250;l,500;r,750')
-    ```
-    Each bound is separated by ';' and each field value in bound is separated by ','. In the example above, we provide 3 bounds to distribute records to 4 partitions. The values 'f','l','r' can evenly distribute the records. Inside carbondata, for a record we compare the value of sort columns with that of the bounds and decide which partition the record will be forwarded to.
-
-    **NOTE:**
-    * SORT_COLUMN_BOUNDS will be used only when the SORT_SCOPE is 'local_sort'.
-    * Carbondata will use these bounds as ranges to process data concurrently during the final sort percedure. The records will be sorted and written out inside each partition. Since the partition is sorted, all records will be sorted.
-    * Since the actual order and literal order of the dictionary column are not necessarily the same, we do not recommend you to use this feature if the first sort column is 'dictionary_include'.
-    * The option works better if your CPU usage during loading is low. If your system is already CPU tense, better not to use this option. Besides, it depends on the user to specify the bounds. If user does not know the exactly bounds to make the data distributed evenly among the bounds, loading performance will still be better than before or at least the same as before.
-    * Users can find more information about this option in the description of PR1953.
-
-  - **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'='#',
-   'HEADER'='false',
-   '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-converts 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 default maximum number of characters per column is 32000. If there are more than 32000 characters in a column, please refer to *String longer than 32000 characters* section.
-  * Since Bad Records Path can be specified in create, load and carbon properties. 
-  Therefore, value specified in load will have the highest priority, and value specified in carbon properties will have the least priority.
-
-   **Bad Records Path:**
-        
-   This property is used to specify the location where bad records would be written.
-        
-   ```
-   TBLPROPERTIES('BAD_RECORDS_PATH'='/opt/badrecords'')
-   ```
-        
-  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')
-  ```
-
-  - **GLOBAL_SORT_PARTITIONS:** If the SORT_SCOPE is defined as GLOBAL_SORT, then user can specify the number of partitions to use while shuffling data for sort using GLOBAL_SORT_PARTITIONS. If it is not configured, or configured less than 1, then it uses the number of map task as reduce task. It is recommended that each reduce task deal with 512MB-1GB data.
-
-  ```
-  OPTIONS('GLOBAL_SORT_PARTITIONS'='2')
-  ```
-
-   NOTE:
-   * GLOBAL_SORT_PARTITIONS should be Integer type, the range is [1,Integer.MaxValue].
-   * It is only used when the SORT_SCOPE is GLOBAL_SORT.
-
-### 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 TABLE <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 TABLE 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 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
-  ```
-   NOTE: Update Complex datatype columns is not supported.
-    
-### 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. 
-  
-  There are several types of compaction.
-  
-  ```
-  ALTER TABLE [db_name.]table_name COMPACT 'MINOR/MAJOR/CUSTOM'
-  ```
-
-  - **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.
-  Configure the property carbon.major.compaction.size with appropriate value in MB.
-  
-  This command merges the specified number of segments into one segment: 
-     
-  ```
-  ALTER TABLE table_name COMPACT 'MAJOR'
-  ```
-  
-  - **Custom Compaction**
-  
-  In Custom compaction, user can directly specify segment ids to be merged into one large segment. 
-  All specified segment ids should exist and be valid, otherwise compaction will fail. 
-  Custom compaction is usually done during the off-peak time. 
-  
-  ```
-  ALTER TABLE table_name COMPACT 'CUSTOM' WHERE SEGMENT.ID IN (2,3,4)
-  ```
-  NOTE: Compaction is unsupported for table containing Complex columns.
-  
-
-  - **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'
-  ```
-   NOTE: Hive partition is not supported on complex datatype columns.
-		
-#### 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, ...)    
-  INSERT INTO INTO TABLE [db_name.]table_name PARTITION (partition_spec) <SELECT STATEMENT>
-  ```
-  
-  Example:
-  ```
-  LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.csv'
-  INTO TABLE locationTable
-  PARTITION (country = 'US', state = 'CA')  
-  INSERT INTO TABLE locationTable
-  PARTITION (country = 'US', state = 'AL')
-  SELECT <columns list excluding partition columns> FROM another_user
-  ```
-
-#### 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.csv'
-  INTO TABLE locationTable          
-  INSERT INTO TABLE locationTable
-  SELECT <columns list excluding partition columns> FROM another_user
-  ```
-
-#### 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, ...)
-  ```
-  
-  Example:
-  ```
-  ALTER TABLE locationTable DROP PARTITION (country = 'US');
-  ```
-  
-#### Insert OVERWRITE
-  
-  This command allows you to insert or load overwrite on a specific 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 feature does not support 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.
-
-## 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 cannot be performed for columns of Complex Data Types.
-  * Columns in the BUCKETCOLUMN parameter must be dimensions. The BUCKETCOLUMN parameter cannot 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 list the segments of CarbonData table.
-
-  ```
-  SHOW [HISTORY] SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
-  ```
-  
-  Example:
-  Show visible segments
-  ```
-  SHOW SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4
-  ```
-  Show all segments, include invisible segments
-  ```
-  SHOW HISTORY 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/44eed099/src/site/markdown/datamap-developer-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/datamap-developer-guide.md b/src/site/markdown/datamap-developer-guide.md
index 31afd34..52c993c 100644
--- a/src/site/markdown/datamap-developer-guide.md
+++ b/src/site/markdown/datamap-developer-guide.md
@@ -3,14 +3,28 @@
 ### Introduction
 DataMap is a data structure that can be used to accelerate certain query of the table. Different DataMap can be implemented by developers. 
 Currently, there are two 2 types of DataMap supported:
-1. IndexDataMap: DataMap that leveraging index to accelerate filter query
-2. MVDataMap: DataMap that leveraging Materialized View to accelerate olap style query, like SPJG query (select, predicate, join, groupby)
+1. IndexDataMap: DataMap that leverages index to accelerate filter query
+2. MVDataMap: DataMap that leverages Materialized View to accelerate olap style query, like SPJG query (select, predicate, join, groupby)
 
 ### DataMap provider
 When user issues `CREATE DATAMAP dm ON TABLE main USING 'provider'`, the corresponding DataMapProvider implementation will be created and initialized. 
 Currently, the provider string can be:
-1. preaggregate: one type of MVDataMap that do pre-aggregate of single table
-2. timeseries: one type of MVDataMap that do pre-aggregate based on time dimension of the table
+1. preaggregate: A type of MVDataMap that do pre-aggregate of single table
+2. timeseries: A type of MVDataMap that do pre-aggregate based on time dimension of the table
 3. class name IndexDataMapFactory  implementation: Developer can implement new type of IndexDataMap by extending IndexDataMapFactory
 
-When user issues `DROP DATAMAP dm ON TABLE main`, the corresponding DataMapProvider interface will be called.
\ No newline at end of file
+When user issues `DROP DATAMAP dm ON TABLE main`, the corresponding DataMapProvider interface will be called.
+
+Details about [DataMap Management](./datamap-management.md#datamap-management) and supported [DSL](./datamap-management.md#overview) are documented [here](./datamap-management.md).
+
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/datamap-management.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/datamap-management.md b/src/site/markdown/datamap-management.md
index b5d1aaa..cc17d69 100644
--- a/src/site/markdown/datamap-management.md
+++ b/src/site/markdown/datamap-management.md
@@ -7,7 +7,7 @@
     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.
@@ -17,6 +17,18 @@
 
 # CarbonData DataMap Management
 
+- [Overview](#overview)
+- [DataMap Management](#datamap-management)
+- [Automatic Refresh](#automatic-refresh)
+- [Manual Refresh](#manual-refresh)
+- [DataMap Catalog](#datamap-catalog)
+- [DataMap Related Commands](#datamap-related-commands)
+  - [Explain](#explain)
+  - [Show DataMap](#show-datamap)
+  - [Compaction on DataMap](#compaction-on-datamap)
+
+
+
 ## Overview
 
 DataMap can be created using following DDL
@@ -36,7 +48,7 @@ Currently, there are 5 DataMap implementations in CarbonData.
 | DataMap Provider | Description                              | DMPROPERTIES                             | Management       |
 | ---------------- | ---------------------------------------- | ---------------------------------------- | ---------------- |
 | preaggregate     | single table pre-aggregate table         | No DMPROPERTY is required                | Automatic        |
-| timeseries       | time dimension rollup table              | event_time, xx_granularity, please refer to [Timeseries DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/timeseries-datamap-guide.md) | Automatic        |
+| timeseries       | time dimension rollup table              | event_time, xx_granularity, please refer to [Timeseries DataMap](./timeseries-datamap-guide.md) | Automatic        |
 | mv               | multi-table pre-aggregate table          | No DMPROPERTY is required                | Manual           |
 | lucene           | lucene indexing for text column          | index_columns to specifying the index columns | Automatic |
 | bloomfilter      | bloom filter for high cardinality column, geospatial column | index_columns to specifying the index columns | Automatic |
@@ -49,7 +61,6 @@ There are two kinds of management semantic for DataMap.
 2. Manual Refresh: Create datamap with `WITH DEFERRED REBUILD` in the statement
 
 **CAUTION:**
-Manual refresh currently only works fine for MV, it has some bugs with other types of datamap in Carbondata 1.4.1, so we block this option for them in this version.
 If user create MV datamap without specifying `WITH DEFERRED REBUILD`, carbondata will give a warning and treat the datamap as deferred rebuild.
 
 ### Automatic Refresh
@@ -138,3 +149,15 @@ This feature applies for preaggregate datamap only
 Running Compaction command (`ALTER TABLE COMPACT`) on main table will **not automatically** compact the pre-aggregate tables created on the main table. User need to run Compaction command separately on each pre-aggregate table to compact them.
 
 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 tables. To further improve the query performance, compaction on pre-aggregate tables can be triggered to merge the segments and files in the pre-aggregate tables.
+
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>


[17/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/datamap-developer-guide.html
----------------------------------------------------------------------
diff --git a/content/datamap-developer-guide.html b/content/datamap-developer-guide.html
index 9ec7a91..073af96 100644
--- a/content/datamap-developer-guide.html
+++ b/content/datamap-developer-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -165,19 +216,31 @@
 <p>DataMap is a data structure that can be used to accelerate certain query of the table. Different DataMap can be implemented by developers.
 Currently, there are two 2 types of DataMap supported:</p>
 <ol>
-<li>IndexDataMap: DataMap that leveraging index to accelerate filter query</li>
-<li>MVDataMap: DataMap that leveraging Materialized View to accelerate olap style query, like SPJG query (select, predicate, join, groupby)</li>
+<li>IndexDataMap: DataMap that leverages index to accelerate filter query</li>
+<li>MVDataMap: DataMap that leverages Materialized View to accelerate olap style query, like SPJG query (select, predicate, join, groupby)</li>
 </ol>
 <h3>
 <a id="datamap-provider" class="anchor" href="#datamap-provider" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DataMap provider</h3>
 <p>When user issues <code>CREATE DATAMAP dm ON TABLE main USING 'provider'</code>, the corresponding DataMapProvider implementation will be created and initialized.
 Currently, the provider string can be:</p>
 <ol>
-<li>preaggregate: one type of MVDataMap that do pre-aggregate of single table</li>
-<li>timeseries: one type of MVDataMap that do pre-aggregate based on time dimension of the table</li>
+<li>preaggregate: A type of MVDataMap that do pre-aggregate of single table</li>
+<li>timeseries: A type of MVDataMap that do pre-aggregate based on time dimension of the table</li>
 <li>class name IndexDataMapFactory  implementation: Developer can implement new type of IndexDataMap by extending IndexDataMapFactory</li>
 </ol>
 <p>When user issues <code>DROP DATAMAP dm ON TABLE main</code>, the corresponding DataMapProvider interface will be called.</p>
+<p>Details about <a href="./datamap-management.html#datamap-management">DataMap Management</a> and supported <a href="./datamap-management.html#overview">DSL</a> are documented <a href="./datamap-management.html">here</a>.</p>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -193,4 +256,4 @@ Currently, the provider string can be:</p>
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/datamap-management.html
----------------------------------------------------------------------
diff --git a/content/datamap-management.html b/content/datamap-management.html
index 50191af..8727594 100644
--- a/content/datamap-management.html
+++ b/content/datamap-management.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -161,6 +212,21 @@
                                     <div>
 <h1>
 <a id="carbondata-datamap-management" class="anchor" href="#carbondata-datamap-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData DataMap Management</h1>
+<ul>
+<li><a href="#overview">Overview</a></li>
+<li><a href="#datamap-management">DataMap Management</a></li>
+<li><a href="#automatic-refresh">Automatic Refresh</a></li>
+<li><a href="#manual-refresh">Manual Refresh</a></li>
+<li><a href="#datamap-catalog">DataMap Catalog</a></li>
+<li>
+<a href="#datamap-related-commands">DataMap Related Commands</a>
+<ul>
+<li><a href="#explain">Explain</a></li>
+<li><a href="#show-datamap">Show DataMap</a></li>
+<li><a href="#compaction-on-datamap">Compaction on DataMap</a></li>
+</ul>
+</li>
+</ul>
 <h2>
 <a id="overview" class="anchor" href="#overview" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Overview</h2>
 <p>DataMap can be created using following DDL</p>
@@ -192,7 +258,7 @@
 <tr>
 <td>timeseries</td>
 <td>time dimension rollup table</td>
-<td>event_time, xx_granularity, please refer to <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/timeseries-datamap-guide.html" target=_blank>Timeseries DataMap</a>
+<td>event_time, xx_granularity, please refer to <a href="./timeseries-datamap-guide.html">Timeseries DataMap</a>
 </td>
 <td>Automatic</td>
 </tr>
@@ -224,7 +290,6 @@
 <li>Manual Refresh: Create datamap with <code>WITH DEFERRED REBUILD</code> in the statement</li>
 </ol>
 <p><strong>CAUTION:</strong>
-Manual refresh currently only works fine for MV, it has some bugs with other types of datamap in Carbondata 1.4.1, so we block this option for them in this version.
 If user create MV datamap without specifying <code>WITH DEFERRED REBUILD</code>, carbondata will give a warning and treat the datamap as deferred rebuild.</p>
 <h3>
 <a id="automatic-refresh" class="anchor" href="#automatic-refresh" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Automatic Refresh</h3>
@@ -293,6 +358,17 @@ Scan Table: default.datamap1_table
 <p>This feature applies for preaggregate datamap only</p>
 <p>Running Compaction command (<code>ALTER TABLE COMPACT</code>) on main table will <strong>not automatically</strong> compact the pre-aggregate tables created on the main table. User need to run Compaction command separately on each pre-aggregate table to compact them.</p>
 <p>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 tables. To further improve the query performance, compaction on pre-aggregate tables can be triggered to merge the segments and files in the pre-aggregate tables.</p>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -308,4 +384,4 @@ Scan Table: default.datamap1_table
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/ddl-of-carbondata.html
----------------------------------------------------------------------
diff --git a/content/ddl-of-carbondata.html b/content/ddl-of-carbondata.html
new file mode 100644
index 0000000..3e15a83
--- /dev/null
+++ b/content/ddl-of-carbondata.html
@@ -0,0 +1,1078 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="carbondata-data-definition-language" class="anchor" href="#carbondata-data-definition-language" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Data Definition Language</h1>
+<p>CarbonData DDL statements are documented here,which includes:</p>
+<ul>
+<li>
+<a href="#create-table">CREATE TABLE</a>
+<ul>
+<li><a href="#dictionary-encoding-configuration">Dictionary Encoding</a></li>
+<li><a href="#inverted-index-configuration">Inverted Index</a></li>
+<li><a href="#sort-columns-configuration">Sort Columns</a></li>
+<li><a href="#sort-scope-configuration">Sort Scope</a></li>
+<li><a href="#table-block-size-configuration">Table Block Size</a></li>
+<li><a href="#table-compaction-configuration">Table Compaction</a></li>
+<li><a href="#streaming">Streaming</a></li>
+<li><a href="#local-dictionary-configuration">Local Dictionary</a></li>
+<li><a href="#caching-minmax-value-for-required-columns">Caching Column Min/Max</a></li>
+<li><a href="#caching-at-block-or-blocklet-level">Caching Level</a></li>
+<li><a href="#support-flat-folder-same-as-hiveparquet">Hive/Parquet folder Structure</a></li>
+<li><a href="#string-longer-than-32000-characters">Extra Long String columns</a></li>
+</ul>
+</li>
+<li><a href="#create-table-as-select">CREATE TABLE AS SELECT</a></li>
+<li>
+<a href="#create-external-table">CREATE EXTERNAL TABLE</a>
+<ul>
+<li><a href="#create-external-table-on-managed-table-data-location">External Table on Transactional table location</a></li>
+<li><a href="#create-external-table-on-non-transactional-table-data-location">External Table on non-transactional table location</a></li>
+</ul>
+</li>
+<li><a href="#create-database">CREATE DATABASE</a></li>
+<li>
+<a href="#table-management">TABLE MANAGEMENT</a>
+<ul>
+<li><a href="#show-table">SHOW TABLE</a></li>
+<li>
+<a href="#alter-table">ALTER TABLE</a>
+<ul>
+<li><a href="#rename-table">RENAME TABLE</a></li>
+<li><a href="#add-columns">ADD COLUMNS</a></li>
+<li><a href="#drop-columns">DROP COLUMNS</a></li>
+<li><a href="#change-data-type">CHANGE DATA TYPE</a></li>
+<li><a href="#merge-index">MERGE INDEXES</a></li>
+<li><a href="#set-and-unset-for-local-dictionary-properties">SET/UNSET Local Dictionary Properties</a></li>
+</ul>
+</li>
+<li><a href="#drop-table">DROP TABLE</a></li>
+<li><a href="#refresh-table">REFRESH TABLE</a></li>
+<li><a href="#table-and-column-comment">COMMENTS</a></li>
+</ul>
+</li>
+<li>
+<a href="#partition">PARTITION</a>
+<ul>
+<li>
+<a href="#standard-partition">STANDARD PARTITION(HIVE)</a>
+<ul>
+<li><a href="#insert-overwrite">INSERT OVERWRITE PARTITION</a></li>
+</ul>
+</li>
+<li>
+<a href="#create-hash-partition-table">CARBONDATA PARTITION</a>
+<ul>
+<li><a href="#create-hash-partition-table">HASH PARTITION</a></li>
+<li><a href="#create-range-partition-table">RANGE PARTITION</a></li>
+<li><a href="#create-list-partition-table">LIST PARTITION</a></li>
+</ul>
+</li>
+<li><a href="#show-partitions">SHOW PARTITIONS</a></li>
+<li><a href="#add-a-new-partition">ADD PARTITION</a></li>
+<li><a href="#split-a-partition">SPLIT PARTITION</a></li>
+<li><a href="#drop-a-partition">DROP PARTITION</a></li>
+</ul>
+</li>
+<li><a href="#bucketing">BUCKETING</a></li>
+</ul>
+<h2>
+<a id="create-table" class="anchor" href="#create-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE</h2>
+<p>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.</p>
+<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type , ...)]
+STORED AS carbondata
+[TBLPROPERTIES (property_name=property_value, ...)]
+[LOCATION 'path']
+</code></pre>
+<p><strong>NOTE:</strong> CarbonData also supports "STORED AS carbondata" and "USING carbondata". Find example code at <a href="https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala" target=_blank>CarbonSessionExample</a> in the CarbonData repo.</p>
+<h3>
+<a id="usage-guidelines" class="anchor" href="#usage-guidelines" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
+<p><strong>Supported properties:</strong> <a href="#dictionary-encoding-configuration">DICTIONARY_INCLUDE</a>,<a href="#inverted-index-configuration">NO_INVERTED_INDEX</a>,<a href="#sort-columns-configuration">SORT_COLUMNS</a>,<a href="#sort-scope-configuration">SORT_SCOPE</a>,<a href="#table-block-size-configuration">TABLE_BLOCKSIZE</a>,<a href="#table-compaction-configuration">MAJOR_COMPACTION_SIZE</a>,</p>
+<p><a href="#table-compaction-configuration">AUTO_LOAD_MERGE</a>,<a href="#table-compaction-configuration">COMPACTION_LEVEL_THRESHOLD</a>,<a href="#table-compaction-configuration">COMPACTION_PRESERVE_SEGMENTS</a>,<a href="#table-compaction-configuration">ALLOWED_COMPACTION_DAYS</a>,</p>
+<p><a href="#streaming">streaming</a>,<a href="#local-dictionary-configuration">LOCAL_DICTIONARY_ENABLE</a>,<a href="#local-dictionary-configuration">LOCAL_DICTIONARY_THRESHOLD</a>,<a href="#local-dictionary-configuration">LOCAL_DICTIONARY_INCLUDE</a>,</p>
+<p><a href="#local-dictionary-configuration">LOCAL_DICTIONARY_EXCLUDE</a>,<a href="#caching-minmax-value-for-required-columns">COLUMN_META_CACHE</a>,<a href="#caching-at-block-or-blocklet-level">CACHE_LEVEL</a>,<a href="#support-flat-folder-same-as-hiveparquet">flat_folder</a>,<a href="#string-longer-than-32000-characters">LONG_STRING_COLUMNS</a>,<a href="#bucketing">BUCKETNUMBER</a>,</p>
+<p><a href="#bucketing">BUCKETCOLUMNS</a></p>
+<p>Following are the guidelines for TBLPROPERTIES, CarbonData's additional table options can be set via carbon.properties.</p>
+<ul>
+<li>
+<h5>
+<a id="dictionary-encoding-configuration" class="anchor" href="#dictionary-encoding-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Dictionary Encoding Configuration</h5>
+<p>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.</p>
+<pre><code>TBLPROPERTIES ('DICTIONARY_INCLUDE'='column1, column2')
+</code></pre>
+</li>
+</ul>
+<pre><code>```
+ NOTE: Dictionary Include/Exclude for complex child columns is not supported.
+</code></pre>
+<ul>
+<li>
+<h5>
+<a id="inverted-index-configuration" class="anchor" href="#inverted-index-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Inverted Index Configuration</h5>
+<p>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.</p>
+<pre><code>TBLPROPERTIES ('NO_INVERTED_INDEX'='column1, column3')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="sort-columns-configuration" class="anchor" href="#sort-columns-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Sort Columns Configuration</h5>
+<p>This property is for users to specify which columns belong to the MDK(Multi-Dimensions-Key) index.</p>
+<ul>
+<li>If users don't specify "SORT_COLUMN" property, by default MDK index be built by using all dimension columns except complex data type column.</li>
+<li>If this property is specified but with empty argument, then the table will be loaded without sort.</li>
+<li>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.</li>
+</ul>
+<pre><code>TBLPROPERTIES ('SORT_COLUMNS'='column1, column3')
+OR
+TBLPROPERTIES ('SORT_COLUMNS'='')
+</code></pre>
+<p>NOTE: Sort_Columns for Complex datatype columns is not supported.</p>
+</li>
+<li>
+<h5>
+<a id="sort-scope-configuration" class="anchor" href="#sort-scope-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Sort Scope Configuration</h5>
+<p>This property is for users to specify the scope of the sort during data load, following are the types of sort scope.</p>
+<ul>
+<li>LOCAL_SORT: It is the default sort scope.</li>
+<li>NO_SORT: It will load the data in unsorted manner, it will significantly increase load performance.</li>
+<li>BATCH_SORT: It increases the load performance but decreases the query performance if identified blocks &gt; parallelism.</li>
+<li>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.</li>
+</ul>
+</li>
+</ul>
+<pre><code>### Example:
+</code></pre>
+<pre><code> 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 ('SORT_COLUMNS'='productName,storeCity',
+                'SORT_SCOPE'='NO_SORT')
+</code></pre>
+<p><strong>NOTE:</strong> CarbonData also supports "using carbondata". Find example code at <a href="https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala" target=_blank>SparkSessionExample</a> in the CarbonData repo.</p>
+<ul>
+<li>
+<h5>
+<a id="table-block-size-configuration" class="anchor" href="#table-block-size-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Table Block Size Configuration</h5>
+<p>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.</p>
+<pre><code>TBLPROPERTIES ('TABLE_BLOCKSIZE'='512')
+</code></pre>
+<p><strong>NOTE:</strong> 512 or 512M both are accepted.</p>
+</li>
+<li>
+<h5>
+<a id="table-compaction-configuration" class="anchor" href="#table-compaction-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Table Compaction Configuration</h5>
+<p>These properties are table level compaction configurations, if not specified, system level configurations in carbon.properties will be used.
+Following are 5 configurations:</p>
+<ul>
+<li>MAJOR_COMPACTION_SIZE: same meaning as carbon.major.compaction.size, size in MB.</li>
+<li>AUTO_LOAD_MERGE: same meaning as carbon.enable.auto.load.merge.</li>
+<li>COMPACTION_LEVEL_THRESHOLD: same meaning as carbon.compaction.level.threshold.</li>
+<li>COMPACTION_PRESERVE_SEGMENTS: same meaning as carbon.numberof.preserve.segments.</li>
+<li>ALLOWED_COMPACTION_DAYS: same meaning as carbon.allowed.compaction.days.</li>
+</ul>
+<pre><code>TBLPROPERTIES ('MAJOR_COMPACTION_SIZE'='2048',
+               'AUTO_LOAD_MERGE'='true',
+               'COMPACTION_LEVEL_THRESHOLD'='5,6',
+               'COMPACTION_PRESERVE_SEGMENTS'='10',
+               'ALLOWED_COMPACTION_DAYS'='5')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="streaming" class="anchor" href="#streaming" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Streaming</h5>
+<p>CarbonData supports streaming ingestion for real-time data. You can create the ?streaming? table using the following table properties.</p>
+<pre><code>TBLPROPERTIES ('streaming'='true')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="local-dictionary-configuration" class="anchor" href="#local-dictionary-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Local Dictionary Configuration</h5>
+</li>
+</ul>
+<p>Columns for which dictionary is not generated needs more storage space and in turn more IO. Also since more data will have to be read during query, query performance also would suffer.Generating dictionary per blocklet for such columns would help in saving storage space and assist in improving query performance as carbondata is optimized for handling dictionary encoded columns more effectively.Generating dictionary internally per blocklet is termed as local dictionary. Please refer to <a href="./file-structure-of-carbondata.html">File structure of Carbondata</a> for understanding about the file structure of carbondata and meaning of terms like blocklet.</p>
+<p>Local Dictionary helps in:</p>
+<ol>
+<li>Getting more compression.</li>
+<li>Filter queries and full scan queries will be faster as filter will be done on encoded data.</li>
+<li>Reducing the store size and memory footprint as only unique values will be stored as part of local dictionary and corresponding data will be stored as encoded data.</li>
+<li>Getting higher IO throughput.</li>
+</ol>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>
+<p>Following Data Types are Supported for Local Dictionary:</p>
+<ul>
+<li>STRING</li>
+<li>VARCHAR</li>
+<li>CHAR</li>
+</ul>
+</li>
+<li>
+<p>Following Data Types are not Supported for Local Dictionary:</p>
+<ul>
+<li>SMALLINT</li>
+<li>INTEGER</li>
+<li>BIGINT</li>
+<li>DOUBLE</li>
+<li>DECIMAL</li>
+<li>TIMESTAMP</li>
+<li>DATE</li>
+<li>BOOLEAN</li>
+</ul>
+</li>
+<li>
+<p>In case of multi-level complex dataType columns, primitive string/varchar/char columns are considered for local dictionary generation.</p>
+</li>
+</ul>
+<p>Local dictionary will have to be enabled explicitly during create table or by enabling the <strong>system property</strong> <em><strong>carbon.local.dictionary.enable</strong></em>. By default, Local Dictionary will be disabled for the carbondata table.</p>
+<p>Local Dictionary can be configured using the following properties during create table command:</p>
+<table>
+<thead>
+<tr>
+<th>Properties</th>
+<th>Default value</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>LOCAL_DICTIONARY_ENABLE</td>
+<td>false</td>
+<td>Whether to enable local dictionary generation. <strong>NOTE:</strong> If this property is defined, it will override the value configured at system level by 'carbon.local.dictionary.enable'</td>
+</tr>
+<tr>
+<td>LOCAL_DICTIONARY_THRESHOLD</td>
+<td>10000</td>
+<td>The maximum cardinality of a column upto which carbondata can try to generate local dictionary (maximum - 100000)</td>
+</tr>
+<tr>
+<td>LOCAL_DICTIONARY_INCLUDE</td>
+<td>string/varchar/char columns</td>
+<td>Columns for which Local Dictionary has to be generated.<strong>NOTE:</strong> Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.</td>
+</tr>
+<tr>
+<td>LOCAL_DICTIONARY_EXCLUDE</td>
+<td>none</td>
+<td>Columns for which Local Dictionary need not be generated.</td>
+</tr>
+</tbody>
+</table>
+<p><strong>Fallback behavior:</strong></p>
+<ul>
+<li>When the cardinality of a column exceeds the threshold, it triggers a fallback and the generated dictionary will be reverted and data loading will be continued without dictionary encoding.</li>
+</ul>
+<p><strong>NOTE:</strong> When fallback is triggered, the data loading performance will decrease as encoded data will be discarded and the actual data is written to the temporary sort files.</p>
+<p><strong>Points to be noted:</strong></p>
+<ol>
+<li>
+<p>Reduce Block size:</p>
+<p>Number of Blocks generated is less in case of Local Dictionary as compression ratio is high. This may reduce the number of tasks launched during query, resulting in degradation of query performance if the pruned blocks are less compared to the number of parallel tasks which can be run. So it is recommended to configure smaller block size which in turn generates more number of blocks.</p>
+</li>
+<li>
+<p>All the page-level data for a blocklet needs to be maintained in memory until all the pages encoded for local dictionary is processed in order to handle fallback. Hence the memory required for local dictionary based table is more and this memory increase is proportional to number of columns.</p>
+</li>
+</ol>
+<h3>
+<a id="example" class="anchor" href="#example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
+<pre><code>CREATE TABLE carbontable(
+          
+            column1 string,
+          
+            column2 string,
+          
+            column3 LONG )
+          
+  STORED BY 'carbondata'
+  TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='true','LOCAL_DICTIONARY_THRESHOLD'='1000',
+  'LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>We recommend to use Local Dictionary when cardinality is high but is distributed across multiple loads</li>
+<li>On a large cluster, decoding data can become a bottleneck for global dictionary as there will be many remote reads. In this scenario, it is better to use Local Dictionary.</li>
+<li>When cardinality is less, but loads are repetitive, it is better to use global dictionary as local dictionary generates multiple dictionary files at blocklet level increasing redundancy.</li>
+</ul>
+<ul>
+<li>
+<h5>
+<a id="caching-minmax-value-for-required-columns" class="anchor" href="#caching-minmax-value-for-required-columns" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Caching Min/Max Value for Required Columns</h5>
+<p>By default, CarbonData caches min and max values of all the columns in schema.  As the load increases, the memory required to hold the min and max values increases considerably. This feature enables you to configure min and max values only for the required columns, resulting in optimized memory usage.</p>
+<p>Following are the valid values for COLUMN_META_CACHE:</p>
+<ul>
+<li>If you want no column min/max values to be cached in the driver.</li>
+</ul>
+<pre><code>COLUMN_META_CACHE=??
+</code></pre>
+<ul>
+<li>If you want only col1 min/max values to be cached in the driver.</li>
+</ul>
+<pre><code>COLUMN_META_CACHE=?col1?
+</code></pre>
+<ul>
+<li>If you want min/max values to be cached in driver for all the specified columns.</li>
+</ul>
+<pre><code>COLUMN_META_CACHE=?col1,col2,col3,??
+</code></pre>
+<p>Columns to be cached can be specified either while creating table or after creation of the table.
+During create table operation; specify the columns to be cached in table properties.</p>
+<p>Syntax:</p>
+<pre><code>CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,?) STORED BY ?carbondata? TBLPROPERTIES (?COLUMN_META_CACHE?=?col1,col2,??)
+</code></pre>
+<p>Example:</p>
+<pre><code>CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES (?COLUMN_META_CACHE?=?name?)
+</code></pre>
+<p>After creation of table or on already created tables use the alter table command to configure the columns to be cached.</p>
+<p>Syntax:</p>
+<pre><code>ALTER TABLE [dbName].tableName SET TBLPROPERTIES (?COLUMN_META_CACHE?=?col1,col2,??)
+</code></pre>
+<p>Example:</p>
+<pre><code>ALTER TABLE employee SET TBLPROPERTIES (?COLUMN_META_CACHE?=?city?)
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="caching-at-block-or-blocklet-level" class="anchor" href="#caching-at-block-or-blocklet-level" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Caching at Block or Blocklet Level</h5>
+<p>This feature allows you to maintain the cache at Block level, resulting in optimized usage of the memory. The memory consumption is high if the Blocklet level caching is maintained as a Block can have multiple Blocklet.</p>
+<p>Following are the valid values for CACHE_LEVEL:</p>
+<p><em>Configuration for caching in driver at Block level (default value).</em></p>
+<pre><code>CACHE_LEVEL= ?BLOCK?
+</code></pre>
+<p><em>Configuration for caching in driver at Blocklet level.</em></p>
+<pre><code>CACHE_LEVEL= ?BLOCKLET?
+</code></pre>
+<p>Cache level can be specified either while creating table or after creation of the table.
+During create table operation specify the cache level in table properties.</p>
+<p>Syntax:</p>
+<pre><code>CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,?) STORED BY ?carbondata? TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
+</code></pre>
+<p>Example:</p>
+<pre><code>CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
+</code></pre>
+<p>After creation of table or on already created tables use the alter table command to configure the cache level.</p>
+<p>Syntax:</p>
+<pre><code>ALTER TABLE [dbName].tableName SET TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
+</code></pre>
+<p>Example:</p>
+<pre><code>ALTER TABLE employee SET TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="support-flat-folder-same-as-hiveparquet" class="anchor" href="#support-flat-folder-same-as-hiveparquet" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Support Flat folder same as Hive/Parquet</h5>
+<p>This feature allows all carbondata and index files to keep directy under tablepath. Currently all carbondata/carbonindex files written under tablepath/Fact/Part0/Segment_NUM folder and it is not same as hive/parquet folder structure. This feature makes all files written will be directly under tablepath, it does not maintain any segment folder structure.This is useful for interoperability between the execution engines and plugin with other execution engines like hive or presto becomes easier.</p>
+<p>Following table property enables this feature and default value is false.</p>
+<pre><code> 'flat_folder'='true'
+</code></pre>
+<p>Example:</p>
+<pre><code>CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES ('flat_folder'='true')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="string-longer-than-32000-characters" class="anchor" href="#string-longer-than-32000-characters" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>String longer than 32000 characters</h5>
+<p>In common scenarios, the length of string is less than 32000,
+so carbondata stores the length of content using Short to reduce memory and space consumption.
+To support string longer than 32000 characters, carbondata introduces a table property called <code>LONG_STRING_COLUMNS</code>.
+For these columns, carbondata internally stores the length of content using Integer.</p>
+<p>You can specify the columns as 'long string column' using below tblProperties:</p>
+<pre><code>// specify col1, col2 as long string columns
+TBLPROPERTIES ('LONG_STRING_COLUMNS'='col1,col2')
+</code></pre>
+<p>Besides, you can also use this property through DataFrame by</p>
+<pre><code>df.format("carbondata")
+  .option("tableName", "carbonTable")
+  .option("long_string_columns", "col1, col2")
+  .save()
+</code></pre>
+<p>If you are using Carbon-SDK, you can specify the datatype of long string column as <code>varchar</code>.
+You can refer to SDKwriterTestCase for example.</p>
+<p><strong>NOTE:</strong> The LONG_STRING_COLUMNS can only be string/char/varchar columns and cannot be dictionary_include/sort_columns/complex columns.</p>
+</li>
+</ul>
+<h2>
+<a id="create-table-as-select" class="anchor" href="#create-table-as-select" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE AS SELECT</h2>
+<p>This function allows user 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.</p>
+<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name 
+STORED BY 'carbondata' 
+[TBLPROPERTIES (key1=val1, key2=val2, ...)] 
+AS select_statement;
+</code></pre>
+<h3>
+<a id="examples" class="anchor" href="#examples" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples</h3>
+<pre><code>carbon.sql("CREATE TABLE source_table(
+                           id INT,
+                           name STRING,
+                           city STRING,
+                           age INT)
+            STORED AS parquet")
+carbon.sql("INSERT INTO source_table SELECT 1,'bob','shenzhen',27")
+carbon.sql("INSERT INTO source_table SELECT 2,'david','shenzhen',31")
+
+carbon.sql("CREATE TABLE target_table
+            STORED BY 'carbondata'
+            AS SELECT city,avg(age) FROM source_table GROUP BY city")
+            
+carbon.sql("SELECT * FROM target_table").show
+  // results:
+  //    +--------+--------+
+  //    |    city|avg(age)|
+  //    +--------+--------+
+  //    |shenzhen|    29.0|
+  //    +--------+--------+
+
+</code></pre>
+<h2>
+<a id="create-external-table" class="anchor" href="#create-external-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE EXTERNAL TABLE</h2>
+<p>This function allows user to create external table by specifying location.</p>
+<pre><code>CREATE EXTERNAL TABLE [IF NOT EXISTS] [db_name.]table_name 
+STORED BY 'carbondata' LOCATION ?$FilesPath?
+</code></pre>
+<h3>
+<a id="create-external-table-on-managed-table-data-location" class="anchor" href="#create-external-table-on-managed-table-data-location" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create external table on managed table data location.</h3>
+<p>Managed table data location provided will have both FACT and Metadata folder.
+This data can be generated by creating a normal carbon table and use this path as $FilesPath in the above syntax.</p>
+<p><strong>Example:</strong></p>
+<pre><code>sql("CREATE TABLE origin(key INT, value STRING) STORED BY 'carbondata'")
+sql("INSERT INTO origin select 100,'spark'")
+sql("INSERT INTO origin select 200,'hive'")
+// creates a table in $storeLocation/origin
+
+sql(s"""
+|CREATE EXTERNAL TABLE source
+|STORED BY 'carbondata'
+|LOCATION '$storeLocation/origin'
+""".stripMargin)
+checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin"))
+</code></pre>
+<h3>
+<a id="create-external-table-on-non-transactional-table-data-location" class="anchor" href="#create-external-table-on-non-transactional-table-data-location" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create external table on Non-Transactional table data location.</h3>
+<p>Non-Transactional table data location will have only carbondata and carbonindex files, there will not be a metadata folder (table status and schema).
+Our SDK module currently support writing data in this format.</p>
+<p><strong>Example:</strong></p>
+<pre><code>sql(
+s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbondata' LOCATION
+|'$writerPath' """.stripMargin)
+</code></pre>
+<p>Here writer path will have carbondata and index files.
+This can be SDK output. Refer <a href="./sdk-guide.html">SDK Guide</a>.</p>
+<p><strong>Note:</strong></p>
+<ol>
+<li>Dropping of the external table should not delete the files present in the location.</li>
+<li>When external table is created on non-transactional table data,
+external table will be registered with the schema of carbondata files.
+If multiple files with different schema is present, exception will be thrown.
+So, If table registered with one schema and files are of different schema,
+suggest to drop the external table and create again to register table with new schema.</li>
+</ol>
+<h2>
+<a id="create-database" class="anchor" href="#create-database" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE DATABASE</h2>
+<p>This function creates a new database. By default the database is created in Carbon store location, but you can also specify custom location.</p>
+<pre><code>CREATE DATABASE [IF NOT EXISTS] database_name [LOCATION path];
+</code></pre>
+<h3>
+<a id="example-1" class="anchor" href="#example-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example</h3>
+<pre><code>CREATE DATABASE carbon LOCATION ?hdfs://name_cluster/dir1/carbonstore?;
+</code></pre>
+<h2>
+<a id="table-management" class="anchor" href="#table-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>TABLE MANAGEMENT</h2>
+<h3>
+<a id="show-table" class="anchor" href="#show-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW TABLE</h3>
+<p>This command can be used to list all the tables in current database or all the tables of a specific database.</p>
+<pre><code>SHOW TABLES [IN db_Name]
+</code></pre>
+<p>Example:</p>
+<pre><code>SHOW TABLES
+OR
+SHOW TABLES IN defaultdb
+</code></pre>
+<h3>
+<a id="alter-table" class="anchor" href="#alter-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>ALTER TABLE</h3>
+<p>The following section introduce the commands to modify the physical or logical state of the existing table(s).</p>
+<ul>
+<li>
+<h5>
+<a id="rename-table" class="anchor" href="#rename-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>RENAME TABLE</h5>
+<p>This command is used to rename the existing table.</p>
+<pre><code>ALTER TABLE [db_name.]table_name RENAME TO new_table_name
+</code></pre>
+<p>Examples:</p>
+<pre><code>ALTER TABLE carbon RENAME TO carbonTable
+OR
+ALTER TABLE test_db.carbon RENAME TO test_db.carbonTable
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="add-columns" class="anchor" href="#add-columns" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>ADD COLUMNS</h5>
+<p>This command is used to add a new column to the existing table.</p>
+<pre><code>ALTER TABLE [db_name.]table_name ADD COLUMNS (col_name data_type,...)
+TBLPROPERTIES('DICTIONARY_INCLUDE'='col_name,...',
+'DEFAULT.VALUE.COLUMN_NAME'='default_value')
+</code></pre>
+<p>Examples:</p>
+<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING)
+</code></pre>
+<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DICTIONARY_INCLUDE'='a1')
+</code></pre>
+<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DEFAULT.VALUE.a1'='10')
+</code></pre>
+<p>NOTE: Add Complex datatype columns is not supported.</p>
+</li>
+</ul>
+<p>Users can specify which columns to include and exclude for local dictionary generation after adding new columns. These will be appended with the already existing local dictionary include and exclude columns of main table respectively.</p>
+<pre><code>   ALTER TABLE carbon ADD COLUMNS (a1 STRING, b1 STRING) TBLPROPERTIES('LOCAL_DICTIONARY_INCLUDE'='a1','LOCAL_DICTIONARY_EXCLUDE'='b1')
+</code></pre>
+<ul>
+<li>
+<h5>
+<a id="drop-columns" class="anchor" href="#drop-columns" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DROP COLUMNS</h5>
+<p>This command is used to delete the existing column(s) in a table.</p>
+<pre><code>ALTER TABLE [db_name.]table_name DROP COLUMNS (col_name, ...)
+</code></pre>
+<p>Examples:</p>
+<pre><code>ALTER TABLE carbon DROP COLUMNS (b1)
+OR
+ALTER TABLE test_db.carbon DROP COLUMNS (b1)
+
+ALTER TABLE carbon DROP COLUMNS (c1,d1)
+</code></pre>
+<p>NOTE: Drop Complex child column is not supported.</p>
+</li>
+<li>
+<h5>
+<a id="change-data-type" class="anchor" href="#change-data-type" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CHANGE DATA TYPE</h5>
+<p>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.</p>
+<pre><code>ALTER TABLE [db_name.]table_name CHANGE col_name col_name changed_column_type
+</code></pre>
+<p>Valid Scenarios</p>
+<ul>
+<li>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.</li>
+<li>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.</li>
+<li>
+<strong>NOTE:</strong> The allowed range is 38,38 (precision, scale) and is a valid upper case scenario which is not resulting in data loss.</li>
+</ul>
+<p>Example1:Changing data type of column a1 from INT to BIGINT.</p>
+<pre><code>ALTER TABLE test_db.carbon CHANGE a1 a1 BIGINT
+</code></pre>
+<p>Example2:Changing decimal precision of column a1 from 10 to 18.</p>
+<pre><code>ALTER TABLE test_db.carbon CHANGE a1 a1 DECIMAL(18,2)
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="merge-index" class="anchor" href="#merge-index" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>MERGE INDEX</h5>
+<p>This command is used to merge all the CarbonData index files (.carbonindex) inside a segment to a single CarbonData index merge file (.carbonindexmerge). This enhances the first query performance.</p>
+<pre><code> ALTER TABLE [db_name.]table_name COMPACT 'SEGMENT_INDEX'
+</code></pre>
+<pre><code>Examples:
+```
+ALTER TABLE test_db.carbon COMPACT 'SEGMENT_INDEX'
+```
+**NOTE:**
+
+* Merge index is not supported on streaming table.
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="set-and-unset-for-local-dictionary-properties" class="anchor" href="#set-and-unset-for-local-dictionary-properties" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SET and UNSET for Local Dictionary Properties</h5>
+<p>When set command is used, all the newly set properties will override the corresponding old properties if exists.</p>
+<p>Example to SET Local Dictionary Properties:</p>
+<pre><code>ALTER TABLE tablename SET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='false','LOCAL_DICTIONARY_THRESHOLD'='1000','LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
+</code></pre>
+<p>When Local Dictionary properties are unset, corresponding default values will be used for these properties.</p>
+<p>Example to UNSET Local Dictionary Properties:</p>
+<pre><code>ALTER TABLE tablename UNSET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE','LOCAL_DICTIONARY_THRESHOLD','LOCAL_DICTIONARY_INCLUDE','LOCAL_DICTIONARY_EXCLUDE')
+</code></pre>
+<p><strong>NOTE:</strong> For old tables, by default, local dictionary is disabled. If user wants local dictionary for these tables, user can enable/disable local dictionary for new data at their discretion.
+This can be achieved by using the alter table set command.</p>
+</li>
+</ul>
+<h3>
+<a id="drop-table" class="anchor" href="#drop-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DROP TABLE</h3>
+<p>This command is used to delete an existing table.</p>
+<pre><code>DROP TABLE [IF EXISTS] [db_name.]table_name
+</code></pre>
+<p>Example:</p>
+<pre><code>DROP TABLE IF EXISTS productSchema.productSalesTable
+</code></pre>
+<h3>
+<a id="refresh-table" class="anchor" href="#refresh-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>REFRESH TABLE</h3>
+<p>This command is used to register Carbon table to HIVE meta store catalogue from existing Carbon table data.</p>
+<pre><code>REFRESH TABLE $db_NAME.$table_NAME
+</code></pre>
+<p>Example:</p>
+<pre><code>REFRESH TABLE dbcarbon.productSalesTable
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>The new database name and the old database name should be same.</li>
+<li>Before executing this command the old table schema and data should be copied into the new database location.</li>
+<li>If the table is aggregate table, then all the aggregate tables should be copied to the new database location.</li>
+<li>For old store, the time zone of the source and destination cluster should be same.</li>
+<li>If old cluster used HIVE meta store to store schema, refresh will not work as schema file does not exist in file system.</li>
+</ul>
+<h3>
+<a id="table-and-column-comment" class="anchor" href="#table-and-column-comment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Table and Column Comment</h3>
+<p>You can provide more information on table by using table comment. Similarly you can provide more information about a particular column using column comment.
+You can see the column comment of an existing table using describe formatted command.</p>
+<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type [COMMENT col_comment], ...)]
+  [COMMENT table_comment]
+STORED BY 'carbondata'
+[TBLPROPERTIES (property_name=property_value, ...)]
+</code></pre>
+<p>Example:</p>
+<pre><code>CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
+                              productNumber Int COMMENT 'unique serial number for product')
+COMMENT ?This is table comment?
+ STORED BY 'carbondata'
+ TBLPROPERTIES ('DICTIONARY_INCLUDE'='productNumber')
+</code></pre>
+<p>You can also SET and UNSET table comment using ALTER command.</p>
+<p>Example to SET table comment:</p>
+<pre><code>ALTER TABLE carbon SET TBLPROPERTIES ('comment'='this table comment is modified');
+</code></pre>
+<p>Example to UNSET table comment:</p>
+<pre><code>ALTER TABLE carbon UNSET TBLPROPERTIES ('comment');
+</code></pre>
+<h2>
+<a id="partition" class="anchor" href="#partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>PARTITION</h2>
+<h3>
+<a id="standard-partition" class="anchor" href="#standard-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>STANDARD PARTITION</h3>
+<p>The partition is similar as spark and hive partition, user can use any column to build partition:</p>
+<h4>
+<a id="create-partition-table" class="anchor" href="#create-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Partition Table</h4>
+<p>This command allows you to create table with partition.</p>
+<pre><code>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, ...)]
+</code></pre>
+<p>Example:</p>
+<pre><code> 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'
+</code></pre>
+<p>NOTE: Hive partition is not supported on complex datatype columns.</p>
+<h4>
+<a id="show-partitions" class="anchor" href="#show-partitions" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Show Partitions</h4>
+<p>This command gets the Hive partition information of the table</p>
+<pre><code>SHOW PARTITIONS [db_name.]table_name
+</code></pre>
+<h4>
+<a id="drop-partition" class="anchor" href="#drop-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Drop Partition</h4>
+<p>This command drops the specified Hive partition only.</p>
+<pre><code>ALTER TABLE table_name DROP [IF EXISTS] PARTITION (part_spec, ...)
+</code></pre>
+<p>Example:</p>
+<pre><code>ALTER TABLE locationTable DROP PARTITION (country = 'US');
+</code></pre>
+<h4>
+<a id="insert-overwrite" class="anchor" href="#insert-overwrite" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Insert OVERWRITE</h4>
+<p>This command allows you to insert or load overwrite on a specific partition.</p>
+<pre><code> INSERT OVERWRITE TABLE table_name
+ PARTITION (column = 'partition_name')
+ select_statement
+</code></pre>
+<p>Example:</p>
+<pre><code>INSERT OVERWRITE TABLE partitioned_user
+PARTITION (country = 'US')
+SELECT * FROM another_user au 
+WHERE au.country = 'US';
+</code></pre>
+<h3>
+<a id="carbondata-partitionhashrangelist----alpha-feature-this-partition-feature-does-not-support-update-and-delete-data" class="anchor" href="#carbondata-partitionhashrangelist----alpha-feature-this-partition-feature-does-not-support-update-and-delete-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CARBONDATA PARTITION(HASH,RANGE,LIST) -- Alpha feature, this partition feature does not support update and delete data.</h3>
+<p>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.</p>
+<h3>
+<a id="create-hash-partition-table" class="anchor" href="#create-hash-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Hash Partition Table</h3>
+<p>This command allows us to create hash partition.</p>
+<pre><code>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' ...)]
+</code></pre>
+<p><strong>NOTE:</strong> N is the number of hash partitions</p>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<h3>
+<a id="create-range-partition-table" class="anchor" href="#create-range-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Range Partition Table</h3>
+<p>This command allows us to create range partition.</p>
+<pre><code>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, ...')]
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>The 'RANGE_INFO' must be defined in ascending order in the table properties.</li>
+<li>The default format for partition column of Date/Timestamp type is yyyy-MM-dd. Alternate formats for Date/Timestamp could be defined in CarbonProperties.</li>
+</ul>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<h3>
+<a id="create-list-partition-table" class="anchor" href="#create-list-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create List Partition Table</h3>
+<p>This command allows us to create list partition.</p>
+<pre><code>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, ...')]
+</code></pre>
+<p><strong>NOTE:</strong> List partition supports list info in one level group.</p>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<h3>
+<a id="show-partitions-1" class="anchor" href="#show-partitions-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Show Partitions</h3>
+<p>The following command is executed to get the partition information of the table</p>
+<pre><code>SHOW PARTITIONS [db_name.]table_name
+</code></pre>
+<h3>
+<a id="add-a-new-partition" class="anchor" href="#add-a-new-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Add a new partition</h3>
+<pre><code>ALTER TABLE [db_name].table_name ADD PARTITION('new_partition')
+</code></pre>
+<h3>
+<a id="split-a-partition" class="anchor" href="#split-a-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Split a partition</h3>
+<pre><code>ALTER TABLE [db_name].table_name SPLIT PARTITION(partition_id) INTO('new_partition1', 'new_partition2'...)
+</code></pre>
+<h3>
+<a id="drop-a-partition" class="anchor" href="#drop-a-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Drop a partition</h3>
+<p>Only drop partition definition, but keep data</p>
+<pre><code>  ALTER TABLE [db_name].table_name DROP PARTITION(partition_id)
+</code></pre>
+<p>Drop both partition definition and data</p>
+<pre><code>ALTER TABLE [db_name].table_name DROP PARTITION(partition_id) WITH DATA
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>Hash partition table is not supported for ADD, SPLIT and DROP commands.</li>
+<li>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.</li>
+</ul>
+<pre><code>SegmentDir/0_batchno0-0-1502703086921.carbonindex
+          ^
+SegmentDir/part-0-0_batchno0-0-1502703086921.carbondata
+                   ^
+</code></pre>
+<p>Here are some useful tips to improve query performance of carbonData partition table:</p>
+<ul>
+<li>The partitioned column can be excluded from SORT_COLUMNS, this will let other columns to do the efficient sorting.</li>
+<li>When writing SQL on a partition table, try to use filters on the partition column.</li>
+</ul>
+<h2>
+<a id="bucketing" class="anchor" href="#bucketing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>BUCKETING</h2>
+<p>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.</p>
+<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
+                  [(col_name data_type, ...)]
+STORED BY 'carbondata'
+TBLPROPERTIES('BUCKETNUMBER'='noOfBuckets',
+'BUCKETCOLUMNS'='columnname')
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>Bucketing cannot be performed for columns of Complex Data Types.</li>
+<li>Columns in the BUCKETCOLUMN parameter must be dimensions. The BUCKETCOLUMN parameter cannot be a measure or a combination of measures and dimensions.</li>
+</ul>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/ddl-operation-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/ddl-operation-on-carbondata.html b/content/ddl-operation-on-carbondata.html
index 444428f..807adf3 100644
--- a/content/ddl-operation-on-carbondata.html
+++ b/content/ddl-operation-on-carbondata.html
@@ -745,4 +745,4 @@ of columns is used.</p>
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>


[19/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
Added new page layout & updated as per new md files


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

Branch: refs/heads/asf-site
Commit: 44eed099cb917e3e4c8ac62cd761566fcef3e35d
Parents: 324588f
Author: Raghunandan S <ca...@gmail.com>
Authored: Tue Aug 28 14:25:59 2018 +0800
Committer: Raghunandan S <ca...@gmail.com>
Committed: Wed Sep 5 21:04:18 2018 +0800

----------------------------------------------------------------------
 carbonscript.sh                                 |    5 +-
 content/WEB-INF/classes/MdFileHandler.class     |  Bin 6144 -> 6144 bytes
 content/WEB-INF/classes/WebsiteLauncher$.class  |  Bin 4209 -> 4248 bytes
 content/WEB-INF/classes/application.conf        |   22 +-
 content/WEB-INF/classes/html/footer.html        |    2 +-
 content/WEB-INF/classes/html/header.html        |    2 +-
 content/bloomfilter-datamap-guide.html          |   68 +-
 content/configuration-parameters.html           |  707 +++++----
 content/css/style.css                           |   76 +-
 content/data-management-on-carbondata.html      |    2 +-
 content/data-management.html                    |    2 +-
 content/datamap-developer-guide.html            |   77 +-
 content/datamap-management.html                 |   86 +-
 content/ddl-of-carbondata.html                  | 1078 ++++++++++++++
 content/ddl-operation-on-carbondata.html        |    2 +-
 content/dml-of-carbondata.html                  |  588 ++++++++
 content/dml-operation-on-carbondata.html        |    2 +-
 content/documentation.html                      |  256 ++++
 content/errorpage.html                          |    2 +-
 content/faq.html                                |  262 +++-
 content/file-structure-of-carbondata.html       |   72 +-
 .../how-to-contribute-to-apache-carbondata.html |  371 +++++
 content/index.html                              |    2 +-
 content/installation-guide.html                 |    2 +-
 content/introduction.html                       |  361 +++++
 content/language-manual.html                    |  273 ++++
 content/lucene-datamap-guide.html               |   68 +-
 content/partition-guide.html                    |    2 +-
 content/pdf/maven-pdf-plugin.pdf                |  Bin 216771 -> 155540 bytes
 content/performance-tuning.html                 |  529 +++++++
 content/preaggregate-datamap-guide.html         |   70 +-
 content/quick-start-guide.html                  |  476 +++++-
 content/release-guide.html                      |   88 +-
 content/s3-guide.html                           |   81 +-
 content/sdk-guide.html                          |  119 +-
 content/sdk-writer-guide.html                   |    2 +-
 content/security.html                           |    2 +-
 content/segment-management-on-carbondata.html   |  326 ++++
 content/streaming-guide.html                    |   68 +-
 content/supported-data-types-in-carbondata.html |   70 +-
 content/timeseries-datamap-guide.html           |   97 +-
 content/troubleshooting.html                    |   63 +-
 content/useful-tips-on-carbondata.html          |   68 +-
 content/videogallery.html                       |    2 +-
 src/main/resources/application.conf             |   18 +-
 src/main/scala/html/header.html                 |   55 +-
 src/main/webapp/bloomfilter-datamap-guide.html  |   68 +-
 src/main/webapp/configuration-parameters.html   |  707 +++++----
 src/main/webapp/css/style.css                   |   76 +-
 .../webapp/data-management-on-carbondata.html   |    2 +-
 src/main/webapp/datamap-developer-guide.html    |   77 +-
 src/main/webapp/datamap-management.html         |   86 +-
 src/main/webapp/ddl-of-carbondata.html          | 1078 ++++++++++++++
 src/main/webapp/dml-of-carbondata.html          |  588 ++++++++
 src/main/webapp/documentation.html              |  256 ++++
 src/main/webapp/errorpage.html                  |    2 +-
 src/main/webapp/faq.html                        |  262 +++-
 .../webapp/file-structure-of-carbondata.html    |   72 +-
 .../how-to-contribute-to-apache-carbondata.html |  371 +++++
 src/main/webapp/index.html                      |    2 +-
 src/main/webapp/installation-guide.html         |    2 +-
 src/main/webapp/introduction.html               |  361 +++++
 src/main/webapp/language-manual.html            |  273 ++++
 src/main/webapp/lucene-datamap-guide.html       |   68 +-
 src/main/webapp/performance-tuning.html         |  529 +++++++
 src/main/webapp/preaggregate-datamap-guide.html |   70 +-
 src/main/webapp/quick-start-guide.html          |  476 +++++-
 src/main/webapp/release-guide.html              |  572 +++++++
 src/main/webapp/s3-guide.html                   |   81 +-
 src/main/webapp/sdk-guide.html                  |  119 +-
 src/main/webapp/sdk-writer-guide.html           |    2 +-
 src/main/webapp/security.html                   |    2 +-
 .../segment-management-on-carbondata.html       |  326 ++++
 src/main/webapp/streaming-guide.html            |   68 +-
 .../supported-data-types-in-carbondata.html     |   70 +-
 src/main/webapp/timeseries-datamap-guide.html   |   97 +-
 src/main/webapp/troubleshooting.html            |   63 +-
 src/main/webapp/useful-tips-on-carbondata.html  |   68 +-
 src/main/webapp/videogallery.html               |    2 +-
 src/site/markdown/bloomfilter-datamap-guide.md  |   14 +-
 src/site/markdown/configuration-parameters.md   |  286 ++--
 .../markdown/data-management-on-carbondata.md   | 1394 ------------------
 src/site/markdown/datamap-developer-guide.md    |   24 +-
 src/site/markdown/datamap-management.md         |   29 +-
 src/site/markdown/ddl-of-carbondata.md          |  952 ++++++++++++
 src/site/markdown/dml-of-carbondata.md          |  456 ++++++
 src/site/markdown/documentation.md              |   70 +
 src/site/markdown/faq.md                        |  276 +++-
 .../markdown/file-structure-of-carbondata.md    |   13 +
 .../how-to-contribute-to-apache-carbondata.md   |  199 +++
 src/site/markdown/installation-guide.md         |  198 ---
 src/site/markdown/introduction.md               |  172 +++
 src/site/markdown/language-manual.md            |   51 +
 src/site/markdown/lucene-datamap-guide.md       |   11 +
 src/site/markdown/performance-tuning.md         |  183 +++
 src/site/markdown/preaggregate-datamap-guide.md |   13 +-
 src/site/markdown/quick-start-guide.md          |  352 ++++-
 src/site/markdown/release-guide.md              |  428 ++++++
 src/site/markdown/s3-guide.md                   |   21 +-
 src/site/markdown/sdk-guide.md                  |   75 +-
 .../segment-management-on-carbondata.md         |  154 ++
 src/site/markdown/streaming-guide.md            |   13 +
 .../supported-data-types-in-carbondata.md       |   16 +-
 src/site/markdown/timeseries-datamap-guide.md   |   43 +-
 src/site/markdown/troubleshooting.md            |  267 ----
 src/site/markdown/useful-tips-on-carbondata.md  |  179 ---
 106 files changed, 16216 insertions(+), 3093 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/carbonscript.sh
----------------------------------------------------------------------
diff --git a/carbonscript.sh b/carbonscript.sh
index b77e42d..df9de3e 100755
--- a/carbonscript.sh
+++ b/carbonscript.sh
@@ -6,4 +6,7 @@ mvn exec:java -Dexec.mainClass=WebsiteLauncher
 
 mvn install
 
-# mvn exec:java -Dexec.mainClass=CleanUp
\ No newline at end of file
+# mvn exec:java -Dexec.mainClass=CleanUp
+
+find . -type f -name "*.html" ! -name "header.html" ! -name "footer.html" -exec sed -i '' 's/&lt;script&gt;/<script>/g' {} +
+find . -type f -name "*.html" ! -name "header.html" ! -name "footer.html" -exec sed -i '' 's/&lt;\/script&gt;/<\/script>/g' {} +

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/WEB-INF/classes/MdFileHandler.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/MdFileHandler.class b/content/WEB-INF/classes/MdFileHandler.class
index 58088b3..f39c098 100644
Binary files a/content/WEB-INF/classes/MdFileHandler.class and b/content/WEB-INF/classes/MdFileHandler.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/WEB-INF/classes/WebsiteLauncher$.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/WebsiteLauncher$.class b/content/WEB-INF/classes/WebsiteLauncher$.class
index fc13342..9e36ad5 100644
Binary files a/content/WEB-INF/classes/WebsiteLauncher$.class and b/content/WEB-INF/classes/WebsiteLauncher$.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/WEB-INF/classes/application.conf
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/application.conf b/content/WEB-INF/classes/application.conf
index ba5b4b5..b5f941f 100644
--- a/content/WEB-INF/classes/application.conf
+++ b/content/WEB-INF/classes/application.conf
@@ -1,16 +1,21 @@
 fileList=["configuration-parameters",
-  "data-management-on-carbondata",
    "quick-start-guide",
   "faq",
   "file-structure-of-carbondata",
-  "installation-guide",
   "streaming-guide",
   "supported-data-types-in-carbondata",
-  "troubleshooting",
-  "useful-tips-on-carbondata",
+  "performance-tuning",
   "sdk-guide",
+  "datamap-developer-guide",
+  "ddl-of-carbondata",
+  "dml-of-carbondata",
+  "documentation",
+  "language-manual",
   "s3-guide",
-  "datamap-developer-guide"
+  "segment-management-on-carbondata",
+  "release-guide",
+  "how-to-contribute-to-apache-carbondata",
+  "introduction"
   ]
 dataMapFileList=[
   "bloomfilter-datamap-guide",
@@ -19,15 +24,14 @@ dataMapFileList=[
   "timeseries-datamap-guide",
   "datamap-management"]
 
-dataMapFilesUrl="https://raw.githubusercontent.com/apache/carbondata/master/docs/datamap/"
+dataMapFilesUrl="file:///Users/aditi_advith/Documents/code/carbondata/docs/datamap/"
 
-fileListToRetain=["quick-start-guide",
-  "installation-guide"
+fileListToRetain=["quick-start-guide"
 ]
 headerPath="src/main/scala/html/header.html"
 footerPath="src/main/scala/html/footer.html"
 outputFileLocation="src/main/webapp/"
-apiUrl="https://raw.githubusercontent.com/apache/carbondata/master/docs/"
+apiUrl="file:///Users/aditi_advith/Documents/code/carbondata/docs/"
 mdLink="https://api.github.com/markdown/raw"
 mdFileLocation="src/site/markdown/"
 imagesFilesList=["file-structure-of-carbondata"]
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/WEB-INF/classes/html/footer.html
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/html/footer.html b/content/WEB-INF/classes/html/footer.html
index 47aa819..eb80d58 100644
--- a/content/WEB-INF/classes/html/footer.html
+++ b/content/WEB-INF/classes/html/footer.html
@@ -13,4 +13,4 @@
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/WEB-INF/classes/html/header.html
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/html/header.html b/content/WEB-INF/classes/html/header.html
index 9e904d0..5d5a2e0 100644
--- a/content/WEB-INF/classes/html/header.html
+++ b/content/WEB-INF/classes/html/header.html
@@ -158,4 +158,4 @@
                         <div id="viewpage" name="viewpage">
                             <div class="row">
                                 <div class="col-sm-12  col-md-12">
-                                    <div>
\ No newline at end of file
+                                    <div>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/bloomfilter-datamap-guide.html
----------------------------------------------------------------------
diff --git a/content/bloomfilter-datamap-guide.html b/content/bloomfilter-datamap-guide.html
index e3b3eb6..54c14fe 100644
--- a/content/bloomfilter-datamap-guide.html
+++ b/content/bloomfilter-datamap-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -305,6 +356,17 @@ but if it can reduce the number of spark task,
 there is still a chance that BloomFilter datamap can enhance the performance for concurrent query.</li>
 <li>Note that BloomFilter datamap will decrease the data loading performance and may cause slightly storage expansion (for datamap index file).</li>
 </ul>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -320,4 +382,4 @@ there is still a chance that BloomFilter datamap can enhance the performance for
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>


[03/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/ddl-of-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/ddl-of-carbondata.md b/src/site/markdown/ddl-of-carbondata.md
new file mode 100644
index 0000000..5535a40
--- /dev/null
+++ b/src/site/markdown/ddl-of-carbondata.md
@@ -0,0 +1,952 @@
+<!--
+    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 Data Definition Language
+
+CarbonData DDL statements are documented here,which includes:
+
+* [CREATE TABLE](#create-table)
+  * [Dictionary Encoding](#dictionary-encoding-configuration)
+  * [Inverted Index](#inverted-index-configuration)
+  * [Sort Columns](#sort-columns-configuration)
+  * [Sort Scope](#sort-scope-configuration)
+  * [Table Block Size](#table-block-size-configuration)
+  * [Table Compaction](#table-compaction-configuration)
+  * [Streaming](#streaming)
+  * [Local Dictionary](#local-dictionary-configuration)
+  * [Caching Column Min/Max](#caching-minmax-value-for-required-columns)
+  * [Caching Level](#caching-at-block-or-blocklet-level)
+  * [Hive/Parquet folder Structure](#support-flat-folder-same-as-hiveparquet)
+  * [Extra Long String columns](#string-longer-than-32000-characters)
+* [CREATE TABLE AS SELECT](#create-table-as-select)
+* [CREATE EXTERNAL TABLE](#create-external-table)
+  * [External Table on Transactional table location](#create-external-table-on-managed-table-data-location)
+  * [External Table on non-transactional table location](#create-external-table-on-non-transactional-table-data-location)
+* [CREATE DATABASE](#create-database)
+* [TABLE MANAGEMENT](#table-management)
+  * [SHOW TABLE](#show-table)
+  * [ALTER TABLE](#alter-table)
+    * [RENAME TABLE](#rename-table)
+    * [ADD COLUMNS](#add-columns)
+    * [DROP COLUMNS](#drop-columns)
+    * [CHANGE DATA TYPE](#change-data-type)
+    * [MERGE INDEXES](#merge-index)
+    * [SET/UNSET Local Dictionary Properties](#set-and-unset-for-local-dictionary-properties)
+  * [DROP TABLE](#drop-table)
+  * [REFRESH TABLE](#refresh-table)
+  * [COMMENTS](#table-and-column-comment)
+* [PARTITION](#partition)
+  * [STANDARD PARTITION(HIVE)](#standard-partition)
+    * [INSERT OVERWRITE PARTITION](#insert-overwrite)
+  * [CARBONDATA PARTITION](#create-hash-partition-table)
+    * [HASH PARTITION](#create-hash-partition-table)
+    * [RANGE PARTITION](#create-range-partition-table)
+    * [LIST PARTITION](#create-list-partition-table)
+  * [SHOW PARTITIONS](#show-partitions)
+  * [ADD PARTITION](#add-a-new-partition)
+  * [SPLIT PARTITION](#split-a-partition)
+  * [DROP PARTITION](#drop-a-partition)
+* [BUCKETING](#bucketing)
+
+## 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 AS carbondata
+  [TBLPROPERTIES (property_name=property_value, ...)]
+  [LOCATION 'path']
+  ```
+  **NOTE:** CarbonData also supports "STORED AS carbondata" and "USING carbondata". Find example code at [CarbonSessionExample](https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala) in the CarbonData repo.
+### Usage Guidelines
+
+**Supported properties:** [DICTIONARY_INCLUDE](#dictionary-encoding-configuration),[NO_INVERTED_INDEX](#inverted-index-configuration),[SORT_COLUMNS](#sort-columns-configuration),[SORT_SCOPE](#sort-scope-configuration),[TABLE_BLOCKSIZE](#table-block-size-configuration),[MAJOR_COMPACTION_SIZE](#table-compaction-configuration),
+
+[AUTO_LOAD_MERGE](#table-compaction-configuration),[COMPACTION_LEVEL_THRESHOLD](#table-compaction-configuration),[COMPACTION_PRESERVE_SEGMENTS](#table-compaction-configuration),[ALLOWED_COMPACTION_DAYS](#table-compaction-configuration),
+
+[streaming](#streaming),[LOCAL_DICTIONARY_ENABLE](#local-dictionary-configuration),[LOCAL_DICTIONARY_THRESHOLD](#local-dictionary-configuration),[LOCAL_DICTIONARY_INCLUDE](#local-dictionary-configuration),
+
+[LOCAL_DICTIONARY_EXCLUDE](#local-dictionary-configuration),[COLUMN_META_CACHE](#caching-minmax-value-for-required-columns),[CACHE_LEVEL](#caching-at-block-or-blocklet-level),[flat_folder](#support-flat-folder-same-as-hiveparquet),[LONG_STRING_COLUMNS](#string-longer-than-32000-characters),[BUCKETNUMBER](#bucketing),
+
+[BUCKETCOLUMNS](#bucketing)
+
+ 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')
+	```
+	 NOTE: Dictionary Include/Exclude for complex child columns is not supported.
+	
+   - ##### 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'='')
+     ```
+     NOTE: Sort_Columns for Complex datatype columns is not supported.
+
+   - ##### 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. 
+	
+	### 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 ('SORT_COLUMNS'='productName,storeCity',
+                   'SORT_SCOPE'='NO_SORT')
+   ```
+
+   **NOTE:** CarbonData also supports "using carbondata". Find example code at [SparkSessionExample](https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala) in the CarbonData repo.
+
+   - ##### 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 as carbon.major.compaction.size, size in MB.
+     * AUTO_LOAD_MERGE: same meaning as carbon.enable.auto.load.merge.
+     * COMPACTION_LEVEL_THRESHOLD: same meaning as carbon.compaction.level.threshold.
+     * COMPACTION_PRESERVE_SEGMENTS: same meaning as carbon.numberof.preserve.segments.
+     * ALLOWED_COMPACTION_DAYS: same meaning as 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')
+     ```
+
+   - ##### Local Dictionary Configuration
+
+   Columns for which dictionary is not generated needs more storage space and in turn more IO. Also since more data will have to be read during query, query performance also would suffer.Generating dictionary per blocklet for such columns would help in saving storage space and assist in improving query performance as carbondata is optimized for handling dictionary encoded columns more effectively.Generating dictionary internally per blocklet is termed as local dictionary. Please refer to [File structure of Carbondata](./file-structure-of-carbondata.md) for understanding about the file structure of carbondata and meaning of terms like blocklet.
+
+   Local Dictionary helps in:
+   1. Getting more compression.
+   2. Filter queries and full scan queries will be faster as filter will be done on encoded data.
+   3. Reducing the store size and memory footprint as only unique values will be stored as part of local dictionary and corresponding data will be stored as encoded data.
+   4. Getting higher IO throughput.
+
+   **NOTE:** 
+
+   * Following Data Types are Supported for Local Dictionary:
+      * STRING
+      * VARCHAR
+      * CHAR
+
+   * Following Data Types are not Supported for Local Dictionary: 
+      * SMALLINT
+      * INTEGER
+      * BIGINT
+      * DOUBLE
+      * DECIMAL
+      * TIMESTAMP
+      * DATE
+      * BOOLEAN
+   
+   * In case of multi-level complex dataType columns, primitive string/varchar/char columns are considered for local dictionary generation.
+
+   Local dictionary will have to be enabled explicitly during create table or by enabling the **system property** ***carbon.local.dictionary.enable***. By default, Local Dictionary will be disabled for the carbondata table.
+    
+   Local Dictionary can be configured using the following properties during create table command: 
+          
+
+| Properties | Default value | Description |
+| ---------- | ------------- | ----------- |
+| LOCAL_DICTIONARY_ENABLE | false | Whether to enable local dictionary generation. **NOTE:** If this property is defined, it will override the value configured at system level by 'carbon.local.dictionary.enable' |
+| LOCAL_DICTIONARY_THRESHOLD | 10000 | The maximum cardinality of a column upto which carbondata can try to generate local dictionary (maximum - 100000) |
+| LOCAL_DICTIONARY_INCLUDE | string/varchar/char columns| Columns for which Local Dictionary has to be generated.**NOTE:** Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.|
+| LOCAL_DICTIONARY_EXCLUDE | none | Columns for which Local Dictionary need not be generated. |
+
+   **Fallback behavior:** 
+
+   * When the cardinality of a column exceeds the threshold, it triggers a fallback and the generated dictionary will be reverted and data loading will be continued without dictionary encoding.
+
+   **NOTE:** When fallback is triggered, the data loading performance will decrease as encoded data will be discarded and the actual data is written to the temporary sort files.
+
+   **Points to be noted:**
+
+   1. Reduce Block size:
+   
+      Number of Blocks generated is less in case of Local Dictionary as compression ratio is high. This may reduce the number of tasks launched during query, resulting in degradation of query performance if the pruned blocks are less compared to the number of parallel tasks which can be run. So it is recommended to configure smaller block size which in turn generates more number of blocks.
+      
+   2. All the page-level data for a blocklet needs to be maintained in memory until all the pages encoded for local dictionary is processed in order to handle fallback. Hence the memory required for local dictionary based table is more and this memory increase is proportional to number of columns. 
+      
+### Example:
+
+   ```
+   CREATE TABLE carbontable(
+             
+               column1 string,
+             
+               column2 string,
+             
+               column3 LONG )
+             
+     STORED BY 'carbondata'
+     TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='true','LOCAL_DICTIONARY_THRESHOLD'='1000',
+     'LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
+   ```
+
+   **NOTE:** 
+
+   * We recommend to use Local Dictionary when cardinality is high but is distributed across multiple loads
+   * On a large cluster, decoding data can become a bottleneck for global dictionary as there will be many remote reads. In this scenario, it is better to use Local Dictionary.
+   * When cardinality is less, but loads are repetitive, it is better to use global dictionary as local dictionary generates multiple dictionary files at blocklet level increasing redundancy.
+
+   - ##### Caching Min/Max Value for Required Columns
+
+     By default, CarbonData caches min and max values of all the columns in schema.  As the load increases, the memory required to hold the min and max values increases considerably. This feature enables you to configure min and max values only for the required columns, resulting in optimized memory usage. 
+
+      Following are the valid values for COLUMN_META_CACHE:
+      * If you want no column min/max values to be cached in the driver.
+
+      ```
+      COLUMN_META_CACHE=’’
+      ```
+
+      * If you want only col1 min/max values to be cached in the driver.
+
+      ```
+      COLUMN_META_CACHE=’col1’
+      ```
+
+      * If you want min/max values to be cached in driver for all the specified columns.
+
+      ```
+      COLUMN_META_CACHE=’col1,col2,col3,…’
+      ```
+
+      Columns to be cached can be specified either while creating table or after creation of the table.
+      During create table operation; specify the columns to be cached in table properties.
+
+      Syntax:
+
+      ```
+      CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,…) STORED BY ‘carbondata’ TBLPROPERTIES (‘COLUMN_META_CACHE’=’col1,col2,…’)
+      ```
+
+      Example:
+
+      ```
+      CREATE TABLE employee (name String, city String, id int) STORED BY ‘carbondata’ TBLPROPERTIES (‘COLUMN_META_CACHE’=’name’)
+      ```
+
+      After creation of table or on already created tables use the alter table command to configure the columns to be cached.
+
+      Syntax:
+
+      ```
+      ALTER TABLE [dbName].tableName SET TBLPROPERTIES (‘COLUMN_META_CACHE’=’col1,col2,…’)
+      ```
+
+      Example:
+
+      ```
+      ALTER TABLE employee SET TBLPROPERTIES (‘COLUMN_META_CACHE’=’city’)
+      ```
+
+   - ##### Caching at Block or Blocklet Level
+
+     This feature allows you to maintain the cache at Block level, resulting in optimized usage of the memory. The memory consumption is high if the Blocklet level caching is maintained as a Block can have multiple Blocklet.
+
+      Following are the valid values for CACHE_LEVEL:
+
+      *Configuration for caching in driver at Block level (default value).*
+
+      ```
+      CACHE_LEVEL= ‘BLOCK’
+      ```
+
+      *Configuration for caching in driver at Blocklet level.*
+
+      ```
+      CACHE_LEVEL= ‘BLOCKLET’
+      ```
+
+      Cache level can be specified either while creating table or after creation of the table.
+      During create table operation specify the cache level in table properties.
+
+      Syntax:
+
+      ```
+      CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,…) STORED BY ‘carbondata’ TBLPROPERTIES (‘CACHE_LEVEL’=’Blocklet’)
+      ```
+
+      Example:
+
+      ```
+      CREATE TABLE employee (name String, city String, id int) STORED BY ‘carbondata’ TBLPROPERTIES (‘CACHE_LEVEL’=’Blocklet’)
+      ```
+
+      After creation of table or on already created tables use the alter table command to configure the cache level.
+
+      Syntax:
+
+      ```
+      ALTER TABLE [dbName].tableName SET TBLPROPERTIES (‘CACHE_LEVEL’=’Blocklet’)
+      ```
+
+      Example:
+
+      ```
+      ALTER TABLE employee SET TBLPROPERTIES (‘CACHE_LEVEL’=’Blocklet’)
+      ```
+
+   - ##### Support Flat folder same as Hive/Parquet
+
+       This feature allows all carbondata and index files to keep directy under tablepath. Currently all carbondata/carbonindex files written under tablepath/Fact/Part0/Segment_NUM folder and it is not same as hive/parquet folder structure. This feature makes all files written will be directly under tablepath, it does not maintain any segment folder structure.This is useful for interoperability between the execution engines and plugin with other execution engines like hive or presto becomes easier.
+
+       Following table property enables this feature and default value is false.
+       ```
+        'flat_folder'='true'
+       ```
+       Example:
+       ```
+       CREATE TABLE employee (name String, city String, id int) STORED BY ‘carbondata’ TBLPROPERTIES ('flat_folder'='true')
+       ```
+
+   - ##### String longer than 32000 characters
+
+     In common scenarios, the length of string is less than 32000,
+     so carbondata stores the length of content using Short to reduce memory and space consumption.
+     To support string longer than 32000 characters, carbondata introduces a table property called `LONG_STRING_COLUMNS`.
+     For these columns, carbondata internally stores the length of content using Integer.
+
+     You can specify the columns as 'long string column' using below tblProperties:
+
+     ```
+     // specify col1, col2 as long string columns
+     TBLPROPERTIES ('LONG_STRING_COLUMNS'='col1,col2')
+     ```
+
+     Besides, you can also use this property through DataFrame by
+     ```
+     df.format("carbondata")
+       .option("tableName", "carbonTable")
+       .option("long_string_columns", "col1, col2")
+       .save()
+     ```
+
+     If you are using Carbon-SDK, you can specify the datatype of long string column as `varchar`.
+     You can refer to SDKwriterTestCase for example.
+
+     **NOTE:** The LONG_STRING_COLUMNS can only be string/char/varchar columns and cannot be dictionary_include/sort_columns/complex columns.
+
+## CREATE TABLE AS SELECT
+  This function allows user 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
+  ```
+  carbon.sql("CREATE TABLE source_table(
+                             id INT,
+                             name STRING,
+                             city STRING,
+                             age INT)
+              STORED AS parquet")
+  carbon.sql("INSERT INTO source_table SELECT 1,'bob','shenzhen',27")
+  carbon.sql("INSERT INTO source_table SELECT 2,'david','shenzhen',31")
+  
+  carbon.sql("CREATE TABLE target_table
+              STORED BY 'carbondata'
+              AS SELECT city,avg(age) FROM source_table GROUP BY city")
+              
+  carbon.sql("SELECT * FROM target_table").show
+    // results:
+    //    +--------+--------+
+    //    |    city|avg(age)|
+    //    +--------+--------+
+    //    |shenzhen|    29.0|
+    //    +--------+--------+
+
+  ```
+
+## CREATE EXTERNAL TABLE
+  This function allows user to create external table by specifying location.
+  ```
+  CREATE EXTERNAL TABLE [IF NOT EXISTS] [db_name.]table_name 
+  STORED BY 'carbondata' LOCATION ‘$FilesPath’
+  ```
+
+### Create external table on managed table data location.
+  Managed table data location provided will have both FACT and Metadata folder. 
+  This data can be generated by creating a normal carbon table and use this path as $FilesPath in the above syntax.
+
+  **Example:**
+  ```
+  sql("CREATE TABLE origin(key INT, value STRING) STORED BY 'carbondata'")
+  sql("INSERT INTO origin select 100,'spark'")
+  sql("INSERT INTO origin select 200,'hive'")
+  // creates a table in $storeLocation/origin
+  
+  sql(s"""
+  |CREATE EXTERNAL TABLE source
+  |STORED BY 'carbondata'
+  |LOCATION '$storeLocation/origin'
+  """.stripMargin)
+  checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin"))
+  ```
+
+### Create external table on Non-Transactional table data location.
+  Non-Transactional table data location will have only carbondata and carbonindex files, there will not be a metadata folder (table status and schema).
+  Our SDK module currently support writing data in this format.
+
+  **Example:**
+  ```
+  sql(
+  s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbondata' LOCATION
+  |'$writerPath' """.stripMargin)
+  ```
+
+  Here writer path will have carbondata and index files.
+  This can be SDK output. Refer [SDK Guide](./sdk-guide.md). 
+
+  **Note:**
+  1. Dropping of the external table should not delete the files present in the location.
+  2. When external table is created on non-transactional table data, 
+    external table will be registered with the schema of carbondata files.
+    If multiple files with different schema is present, exception will be thrown.
+    So, If table registered with one schema and files are of different schema, 
+    suggest to drop the external table and create again to register table with new schema.  
+
+
+## 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”;
+  ```
+
+## 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 TABLE
+
+  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 carbonTable
+     OR
+     ALTER TABLE test_db.carbon RENAME TO test_db.carbonTable
+     ```
+
+   - ##### 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')
+     ```
+      NOTE: Add Complex datatype columns is not supported.
+
+Users can specify which columns to include and exclude for local dictionary generation after adding new columns. These will be appended with the already existing local dictionary include and exclude columns of main table respectively.
+  ```
+     ALTER TABLE carbon ADD COLUMNS (a1 STRING, b1 STRING) TBLPROPERTIES('LOCAL_DICTIONARY_INCLUDE'='a1','LOCAL_DICTIONARY_EXCLUDE'='b1')
+  ```
+
+   - ##### 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)
+     ```
+     NOTE: Drop Complex child column is not supported.
+
+   - ##### 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)
+     ```
+- ##### MERGE INDEX
+
+     This command is used to merge all the CarbonData index files (.carbonindex) inside a segment to a single CarbonData index merge file (.carbonindexmerge). This enhances the first query performance.
+     ```
+      ALTER TABLE [db_name.]table_name COMPACT 'SEGMENT_INDEX'
+     ```
+
+      Examples:
+      ```
+      ALTER TABLE test_db.carbon COMPACT 'SEGMENT_INDEX'
+      ```
+      **NOTE:**
+
+      * Merge index is not supported on streaming table.
+
+- ##### SET and UNSET for Local Dictionary Properties
+
+   When set command is used, all the newly set properties will override the corresponding old properties if exists.
+  
+   Example to SET Local Dictionary Properties:
+    ```
+   ALTER TABLE tablename SET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='false','LOCAL_DICTIONARY_THRESHOLD'='1000','LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
+    ```
+   When Local Dictionary properties are unset, corresponding default values will be used for these properties.
+   
+   Example to UNSET Local Dictionary Properties:
+    ```
+   ALTER TABLE tablename UNSET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE','LOCAL_DICTIONARY_THRESHOLD','LOCAL_DICTIONARY_INCLUDE','LOCAL_DICTIONARY_EXCLUDE')
+    ```
+   
+   **NOTE:** For old tables, by default, local dictionary is disabled. If user wants local dictionary for these tables, user can enable/disable local dictionary for new data at their discretion. 
+   This can be achieved by using the alter table set command.
+
+### 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 used HIVE meta store to store schema, refresh will not work as schema file does not exist in file system.
+
+### Table and Column Comment
+
+  You can provide more information on table by using table comment. Similarly you can provide more information about a particular column using column comment. 
+  You can see the column comment of an existing table using describe formatted command.
+
+  ```
+  CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type [COMMENT col_comment], ...)]
+    [COMMENT table_comment]
+  STORED BY 'carbondata'
+  [TBLPROPERTIES (property_name=property_value, ...)]
+  ```
+
+  Example:
+  ```
+  CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
+                                productNumber Int COMMENT 'unique serial number for product')
+  COMMENT “This is table comment”
+   STORED BY 'carbondata'
+   TBLPROPERTIES ('DICTIONARY_INCLUDE'='productNumber')
+  ```
+  You can also SET and UNSET table comment using ALTER command.
+
+  Example to SET table comment:
+
+  ```
+  ALTER TABLE carbon SET TBLPROPERTIES ('comment'='this table comment is modified');
+  ```
+
+  Example to UNSET table comment:
+
+  ```
+  ALTER TABLE carbon UNSET TBLPROPERTIES ('comment');
+  ```
+
+## 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'
+  ```
+   NOTE: Hive partition is not supported on complex datatype columns.
+		
+
+#### 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, ...)
+  ```
+
+  Example:
+  ```
+  ALTER TABLE locationTable DROP PARTITION (country = 'US');
+  ```
+
+#### Insert OVERWRITE
+
+  This command allows you to insert or load overwrite on a specific 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 feature does not support 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.
+
+## 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 cannot be performed for columns of Complex Data Types.
+  * Columns in the BUCKETCOLUMN parameter must be dimensions. The BUCKETCOLUMN parameter cannot 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')
+  ```
+
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/dml-of-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/dml-of-carbondata.md b/src/site/markdown/dml-of-carbondata.md
new file mode 100644
index 0000000..de23f5b
--- /dev/null
+++ b/src/site/markdown/dml-of-carbondata.md
@@ -0,0 +1,456 @@
+<!--
+    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 Data Manipulation Language
+
+CarbonData DML statements are documented here,which includes:
+
+* [LOAD DATA](#load-data)
+* [INSERT DATA](#insert-data-into-carbondata-table)
+* [Load Data Using Static Partition](#load-data-using-static-partition)
+* [Load Data Using Dynamic Partition](#load-data-using-dynamic-partition)
+* [UPDATE AND DELETE](#update-and-delete)
+* [COMPACTION](#compaction)
+* [SEGMENT MANAGEMENT](./segment-management-on-carbondata.md)
+
+
+## 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. 
+
+  ```
+  LOAD DATA [LOCAL] INPATH 'folder_path' 
+  INTO TABLE [db_name.]table_name 
+  OPTIONS(property_name=property_value, ...)
+  ```
+
+  **Supported Properties:** [DELIMITER](#delimiter), [QUOTECHAR](#quotechar), [COMMENTCHAR](#commentchar), [HEADER](#header), [FILEHEADER](#fileheader), [MULTILINE](#multiline), [ESCAPECHAR](#escapechar), [SKIP_EMPTY_LINE](#skip_empty_line), [COMPLEX_DELIMITER_LEVEL_1](#complex_delimiter_level_1), [COMPLEX_DELIMITER_LEVEL_2](#complex_delimiter_level_2), [ALL_DICTIONARY_PATH](#all_dictionary_path), [COLUMNDICT](#columndict), [DATEFORMAT](#dateformat),[ TIMESTAMPFORMAT](#timestampformat), [SORT_COLUMN_BOUNDS](#sort-column-bounds), [SINGLE_PASS](#single_pass), [BAD_RECORDS_LOGGER_ENABLE](#bad-records-handling), [BAD_RECORD_PATH](#bad-records-handling), [BAD_RECORDS_ACTION](#bad-records-handling), [IS_EMPTY_DATA_BAD_RECORD](#bad-records-handling), [GLOBAL_SORT_PARTITIONS](#global_sort_partitions)
+
+  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'='#')
+    ```
+
+  - ##### HEADER:
+    When you load the CSV file without the file header and the file header is the same with the table schema, then add 'HEADER'='false' to load data SQL as user need not provide the file header. By default the value is 'true'.
+    false: CSV file is without file header.
+    true: CSV file is with file header.
+
+    ```
+    OPTIONS('HEADER'='false') 
+    ```
+
+    **NOTE:** If the HEADER option exist and is set to 'true', then the FILEHEADER option is not required.
+
+  - ##### 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 in CSV files.
+
+    ```
+    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).
+
+  - ##### SORT COLUMN BOUNDS:
+
+    Range bounds for sort columns.
+
+    Suppose the table is created with 'SORT_COLUMNS'='name,id' and the range for name is aaa~zzz, the value range for id is 0~1000. Then during data loading, we can specify the following option to enhance data loading performance.
+    ```
+    OPTIONS('SORT_COLUMN_BOUNDS'='f,250;l,500;r,750')
+    ```
+    Each bound is separated by ';' and each field value in bound is separated by ','. In the example above, we provide 3 bounds to distribute records to 4 partitions. The values 'f','l','r' can evenly distribute the records. Inside carbondata, for a record we compare the value of sort columns with that of the bounds and decide which partition the record will be forwarded to.
+
+    **NOTE:**
+    * SORT_COLUMN_BOUNDS will be used only when the SORT_SCOPE is 'local_sort'.
+    * Carbondata will use these bounds as ranges to process data concurrently during the final sort percedure. The records will be sorted and written out inside each partition. Since the partition is sorted, all records will be sorted.
+    * Since the actual order and literal order of the dictionary column are not necessarily the same, we do not recommend you to use this feature if the first sort column is 'dictionary_include'.
+    * The option works better if your CPU usage during loading is low. If your system is already CPU tense, better not to use this option. Besides, it depends on the user to specify the bounds. If user does not know the exactly bounds to make the data distributed evenly among the bounds, loading performance will still be better than before or at least the same as before.
+    * Users can find more information about this option in the description of PR1953.
+
+  - ##### 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'='#',
+   'HEADER'='false',
+   '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-converts 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 default maximum number of characters per column is 32000. If there are more than 32000 characters in a column, please refer to *String longer than 32000 characters* section.
+  * Since Bad Records Path can be specified in create, load and carbon properties. 
+    Therefore, value specified in load will have the highest priority, and value specified in carbon properties will have the least priority.
+
+   **Bad Records Path:**
+         This property is used to specify the location where bad records would be written.
+        
+
+   ```
+   TBLPROPERTIES('BAD_RECORDS_PATH'='/opt/badrecords'')
+   ```
+
+  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')
+  ```
+
+  - ##### GLOBAL_SORT_PARTITIONS:
+
+    If the SORT_SCOPE is defined as GLOBAL_SORT, then user can specify the number of partitions to use while shuffling data for sort using GLOBAL_SORT_PARTITIONS. If it is not configured, or configured less than 1, then it uses the number of map task as reduce task. It is recommended that each reduce task deal with 512MB-1GB data.
+
+  ```
+  OPTIONS('GLOBAL_SORT_PARTITIONS'='2')
+  ```
+
+   NOTE:
+   * GLOBAL_SORT_PARTITIONS should be Integer type, the range is [1,Integer.MaxValue].
+   * It is only used when the SORT_SCOPE is GLOBAL_SORT.
+
+### 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 TABLE <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 TABLE table1 SELECT * FROM TABLE2
+  ```
+
+### 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, ...)    
+  INSERT INTO INTO TABLE [db_name.]table_name PARTITION (partition_spec) <SELECT STATEMENT>
+  ```
+
+  Example:
+  ```
+  LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.csv'
+  INTO TABLE locationTable
+  PARTITION (country = 'US', state = 'CA')  
+  INSERT INTO TABLE locationTable
+  PARTITION (country = 'US', state = 'AL')
+  SELECT <columns list excluding partition columns> FROM another_user
+  ```
+
+### 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.csv'
+  INTO TABLE locationTable          
+  INSERT INTO TABLE locationTable
+  SELECT <columns list excluding partition columns> FROM another_user
+  ```
+
+## 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 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
+  ```
+   NOTE: Update Complex datatype columns is not supported.
+    
+### 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. 
+
+  There are several types of compaction.
+
+  ```
+  ALTER TABLE [db_name.]table_name COMPACT 'MINOR/MAJOR/CUSTOM'
+  ```
+
+  - **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.
+  Configure the property carbon.major.compaction.size with appropriate value in MB.
+
+  This command merges the specified number of segments into one segment: 
+     
+  ```
+  ALTER TABLE table_name COMPACT 'MAJOR'
+  ```
+
+  - **Custom Compaction**
+
+  In Custom compaction, user can directly specify segment ids to be merged into one large segment. 
+  All specified segment ids should exist and be valid, otherwise compaction will fail. 
+  Custom compaction is usually done during the off-peak time. 
+
+  ```
+  ALTER TABLE table_name COMPACT 'CUSTOM' WHERE SEGMENT.ID IN (2,3,4)
+  ```
+  NOTE: Compaction is unsupported for table containing Complex columns.
+
+
+  - **CLEAN SEGMENTS AFTER Compaction**
+
+  Clean the segments which are compacted:
+  ```
+  CLEAN FILES FOR TABLE carbon_table
+  ```
+
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/documentation.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/documentation.md b/src/site/markdown/documentation.md
new file mode 100644
index 0000000..66493bd
--- /dev/null
+++ b/src/site/markdown/documentation.md
@@ -0,0 +1,70 @@
+<!--
+    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.
+-->
+
+# Apache CarbonData Documentation
+
+
+
+Apache CarbonData is a new big data file format for faster interactive query using advanced columnar storage, index, compression and encoding techniques to improve computing efficiency, which helps in speeding up queries by an order of magnitude faster over PetaBytes of data.
+
+
+
+## Getting Started
+
+**File Format Concepts:** Start with the basics of understanding the [CarbonData file format](./file-structure-of-carbondata.md#carbondata-file-structure) and its storage structure.This will help to understand other parts of the documentation, incuding deployment, programming and usage guides. 
+
+**Quick Start:** [Run an example program](./quick-start-guide.md#installing-and-configuring-carbondata-to-run-locally-with-spark-shell) on your local machine or [study some examples](https://github.com/apache/carbondata/tree/master/examples/spark2/src/main/scala/org/apache/carbondata/examples).
+
+**CarbonData SQL Language Reference:** CarbonData extends the Spark SQL language and adds several [DDL](./ddl-of-carbondata.md) and [DML](./dml-of-carbondata.md) statements to support operations on it.Refer to the [Reference Manual](./language-manual.md) to understand the supported features and functions.
+
+**Programming Guides:** You can read our guides about [APIs supported](./sdk-guide.md) to learn how to integrate CarbonData with your applications.
+
+
+
+## Deployment
+
+CarbonData can be integrated with popular Execution engines like [Spark](./quick-start-guide.md#spark) and [Presto](./quick-start-guide.md#presto).Refer to the [Installation and Configuration](./quick-start-guide.md##deployment-modes) section to understand all modes of Integrating CarbonData.
+
+
+
+## Contributing to CarbonData
+
+The Apache CarbonData community welcomes all kinds of contributions from anyone with a passion for
+faster data format.Contributing to CarbonData doesn’t just mean writing code. Helping new users on the mailing list, testing releases, and improving documentation are also welcome.Please follow the [Contributing to CarbonData guidelines](./how-to-contribute-to-apache-carbondata.md) before proposing a design or code change.
+
+
+
+**Compiling CarbonData:** This [guide](https://github.com/apache/carbondata/tree/master/build) will help you to compile and generate the jars for test.
+
+
+
+## External Resources
+
+**Wiki:** You can read the [Apache CarbonData wiki](https://cwiki.apache.org/confluence/display/CARBONDATA/CarbonData+Home) page for upcoming release plan, blogs and training materials.
+
+**Summit:** Presentations from past summits and conferences can be found [here](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609).
+
+**Blogs:** Blogs by external users can be found [here](https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=67635497).
+
+**Performance reports:** TPC-H performance reports can be found [here](https://cwiki.apache.org/confluence/display/CARBONDATA/Performance+-+TPCH+Report+of+CarbonData+%281.2+version%29+and+Parquet+on+Spark+Execution+Engine).
+
+**Trainings:** Training records on design and code flows can be found [here](https://cwiki.apache.org/confluence/display/CARBONDATA/CarbonData+Training+Materials).
+
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__intro').addClass('selected'); });
+</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/faq.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/faq.md b/src/site/markdown/faq.md
index 9f74842..fdc2ca6 100644
--- a/src/site/markdown/faq.md
+++ b/src/site/markdown/faq.md
@@ -7,7 +7,7 @@
     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.
@@ -29,7 +29,24 @@
 * [Why all executors are showing success in Spark UI even after Dataload command failed at Driver side?](#Why-all-executors-are-showing-success-in-Spark-UI-even-after-Dataload-command-failed-at-driver-side)
 * [Why different time zone result for select query output when query SDK writer output?](#Why-different-time-zone-result-for-select-query-output-when-query-SDK-writer-output)
 
+# TroubleShooting
+
+- [Getting tablestatus.lock issues When loading data](#Getting-tablestatus.lock-issues-When-loading-data)
+- [Failed to load thrift libraries](#failed-to-load-thrift-libraries)
+- [Failed to launch the Spark Shell](#failed-to-launch-the-spark-shell)
+- [Failed to execute load query on cluster](#failed-to-execute-load-query-on-cluster)
+- [Failed to execute insert query on cluster](#failed-to-execute-insert-query-on-cluster)
+- [Failed to connect to hiveuser with thrift](#failed-to-connect-to-hiveuser-with-thrift)
+- [Failed to read the metastore db during table](#failed-to-read-the-metastore-db-during-table)
+- [Failed to load data on the cluster](#failed-to-load-data-on-the-cluster)
+- [Failed to insert data on the cluster](#failed-to-insert-data-on-the-cluster)
+- [Failed to execute Concurrent Operations(Load,Insert,Update) on table by multiple workers](#failed-to-execute-concurrent-operations-on-table-by-multiple-workers)
+- [Failed to create a table with a single numeric column](#failed-to-create-a-table-with-a-single-numeric-column)
+
+## 
+
 ## 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?
@@ -195,3 +212,260 @@ cluster timezone is Asia/Shanghai
 TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
 ```
 
+
+
+## Getting tablestatus.lock issues When loading data
+
+  **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 exception
+   ```
+
+  **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 exception
+   ```
+
+  **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 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.
+
+
+
+
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__faq').addClass('selected'); });
+</script>
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/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
index ef31ffd..80c91d5 100644
--- a/src/site/markdown/file-structure-of-carbondata.md
+++ b/src/site/markdown/file-structure-of-carbondata.md
@@ -38,3 +38,16 @@ The file footer can be read once to build the indices in memory, which can be ut
 * 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.
+
+
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/how-to-contribute-to-apache-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/how-to-contribute-to-apache-carbondata.md b/src/site/markdown/how-to-contribute-to-apache-carbondata.md
new file mode 100644
index 0000000..741e6d6
--- /dev/null
+++ b/src/site/markdown/how-to-contribute-to-apache-carbondata.md
@@ -0,0 +1,199 @@
+<!--
+    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.
+-->
+
+# How to contribute to Apache CarbonData
+
+The Apache CarbonData community welcomes all kinds of contributions from anyone with a passion for
+faster data format! Apache CarbonData is a new file format for faster interactive query using
+advanced columnar storage, index, compression and encoding techniques to improve computing
+efficiency,in turn it will help speedup queries an order of magnitude faster over PetaBytes of data.
+
+We use a review-then-commit workflow in CarbonData for all contributions.
+
+* Engage -> Design -> Code -> Review -> Commit
+
+## Engage
+
+### Mailing list(s)
+
+We discuss design and implementation issues on dev@carbondata.apache.org Join by
+emailing dev-subscribe@carbondata.apache.org
+
+### Apache JIRA
+
+We use [Apache JIRA](https://issues.apache.org/jira/browse/CARBONDATA) as an issue tracking and
+project management tool, as well as a way to communicate among a very diverse and distributed set
+of contributors. To be able to gather feedback, avoid frustration, and avoid duplicated efforts all
+CarbonData-related work should be tracked there.
+
+If you do not already have an Apache JIRA account, sign up [here](https://issues.apache.org/jira/).
+
+If a quick search doesn’t turn up an existing JIRA issue for the work you want to contribute,
+create it. Please discuss your proposal with a committer or the component lead in JIRA or,
+alternatively, on the developer mailing list(dev@carbondata.apache.org).
+
+If there’s an existing JIRA issue for your intended contribution, please comment about your
+intended work. Once the work is understood, a committer will assign the issue to you.
+(If you don’t have a JIRA role yet, you’ll be added to the “contributor” role.) If an issue is
+currently assigned, please check with the current assignee before reassigning.
+
+For moderate or large contributions, you should not start coding or writing a design doc unless
+there is a corresponding JIRA issue assigned to you for that work. Simple changes,
+like fixing typos, do not require an associated issue.
+
+### Design
+
+To clearly express your thoughts and get early feedback from other community members, we encourage you to clearly scope, document the design of non-trivial contributions and discuss with the CarbonData community before you start coding.
+
+Generally, the JIRA issue is the best place to gather relevant design docs, comments, or references. It’s great to explicitly include relevant stakeholders early in the conversation. For designs that may be generally interesting, we also encourage conversations on the developer’s mailing list.
+
+### Code
+
+We use GitHub’s pull request functionality to review proposed code changes.
+If you do not already have a personal GitHub account, sign up [here](https://github.com).
+
+### Git config
+
+Ensure to finish the below config(user.email, user.name) before starting PR works.
+```
+$ git config --global user.email "you@example.com"
+$ git config --global user.name "Your Name"
+```
+
+#### Fork the repository on GitHub
+
+Go to the [Apache CarbonData GitHub mirror](https://github.com/apache/carbondata) and
+fork the repository to your account.
+This will be your private workspace for staging changes.
+
+#### Clone the repository locally
+
+You are now ready to create the development environment on your local machine.
+Clone CarbonData’s read-only GitHub mirror.
+```
+$ git clone https://github.com/apache/carbondata.git
+$ cd carbondata
+```
+Add your forked repository as an additional Git remote, where you’ll push your changes.
+```
+$ git remote add <GitHub_user> https://github.com/<GitHub_user>/carbondata.git
+```
+You are now ready to start developing!
+
+#### Create a branch in your fork
+
+You’ll work on your contribution in a branch in your own (forked) repository. Create a local branch,
+initialized with the state of the branch you expect your changes to be merged into.
+Keep in mind that we use several branches, including master, feature-specific, and
+release-specific branches. If you are unsure, initialize with the state of the master branch.
+```
+$ git fetch --all
+$ git checkout -b <my-branch> origin/master
+```
+At this point, you can start making and committing changes to this branch in a standard way.
+
+#### Syncing and pushing your branch
+
+Periodically while you work, and certainly before submitting a pull request, you should update
+your branch with the most recent changes to the target branch.
+```
+$ git pull --rebase
+```
+Remember to always use --rebase parameter to avoid extraneous merge commits.
+
+To push your local, committed changes to your (forked) repository on GitHub, run:
+```
+$ git push <GitHub_user> <my-branch>
+```
+#### Testing
+
+All code should have appropriate unit testing coverage. New code should have new tests in the
+same contribution. Bug fixes should include a regression test to prevent the issue from reoccurring.
+
+For contributions to the Java code, run unit tests locally via Maven.
+```
+$ mvn clean verify
+```
+
+### Review
+
+Once the initial code is complete and the tests pass, it’s time to start the code review process.
+We review and discuss all code, no matter who authors it. It’s a great way to build community,
+since you can learn from other developers, and they become familiar with your contribution.
+It also builds a strong project by encouraging a high quality bar and keeping code consistent
+throughout the project.
+
+#### Create a pull request
+
+Organize your commits to make your reviewer’s job easier. Use the following command to
+re-order, squash, edit, or change description of individual commits.
+```
+$ git rebase -i origin/master
+```
+Navigate to the CarbonData GitHub mirror to create a pull request. The title of the pull request
+should be strictly in the following format:
+```
+[CARBONDATA-JiraTicketNumer][FeatureName] Description of pull request    
+```
+Please include a descriptive pull request message to help make the reviewer’s job easier:
+```
+ - The root cause/problem statement
+ - What is the implemented solution
+ ```
+
+If you know a good committer to review your pull request, please make a comment like the following.
+If not, don’t worry, a committer will pick it up.
+```
+Hi @<committer/reviewer name>, can you please take a look?
+```
+
+#### Code Review and Revision
+
+During the code review process, don’t rebase your branch or otherwise modify published commits,
+since this can remove existing comment history and be confusing to the reviewer,
+When you make a revision, always push it in a new commit.
+
+Our GitHub mirror automatically provides pre-commit testing coverage using Jenkins.
+Please make sure those tests pass,the contribution cannot be merged otherwise.
+
+#### LGTM
+Once the reviewer is happy with the change, they’ll respond with an LGTM (“looks good to me!”).
+At this point, the committer will take over, possibly make some additional touch ups,
+and merge your changes into the codebase.
+
+In the case both the author and the reviewer are committers, either can merge the pull request.
+Just be sure to communicate clearly whose responsibility it is in this particular case.
+
+Thank you for your contribution to Apache CarbonData!
+
+#### Deleting your branch(optional)
+Once the pull request is merged into the Apache CarbonData repository, you can safely delete the
+branch locally and purge it from your forked repository.
+
+From another local branch, run:
+```
+$ git fetch --all
+$ git branch -d <my-branch>
+$ git push <GitHub_user> --delete <my-branch>
+```
+
+
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__contri').addClass('selected'); });
+</script>
+


[38/39] carbondata-site git commit: refactor document

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/bee56334/content/data-management.html
----------------------------------------------------------------------
diff --git a/content/data-management.html b/content/data-management.html
new file mode 100644
index 0000000..93528d8
--- /dev/null
+++ b/content/data-management.html
@@ -0,0 +1,413 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
+                                   target="_blank">Apache CarbonData 1.2.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
+                                   target="_blank">Apache CarbonData 1.1.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
+                                   target="_blank">Apache CarbonData 1.1.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
+                                   target="_blank">Apache CarbonData 1.0.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.2.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.1</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="row">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="data-management" class="anchor" href="#data-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Management</h1>
+<p>This tutorial is going to introduce you to the conceptual details of data management like:</p>
+<ul>
+<li><a href="#loading-data">Loading Data</a></li>
+<li><a href="#deleting-data">Deleting Data</a></li>
+<li><a href="#compacting-data">Compacting Data</a></li>
+<li><a href="#updating-data">Updating Data</a></li>
+</ul>
+<h2>
+<a id="loading-data" class="anchor" href="#loading-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Loading Data</h2>
+<ul>
+<li>
+<p><strong>Scenario</strong></p>
+<p>After creating a table, you can load data to the table using the <a href="dml-operation-on-carbondata.html">LOAD DATA</a> command. The loaded data is available for querying.
+When data load is triggered, the data is encoded in CarbonData format and copied into HDFS CarbonData store path (specified in carbon.properties file)
+in compressed, multi dimensional columnar format for quick analysis queries. The same command can be used to load new data or to
+update the existing data. Only one data load can be triggered for one table. The high cardinality columns of the dictionary encoding are
+automatically recognized and these columns will not be used for dictionary encoding.</p>
+</li>
+<li>
+<p><strong>Procedure</strong></p>
+<p>Data loading is a process that involves execution of multiple steps to read, sort and encode the data in CarbonData store format.
+Each step is executed on different threads. After data loading process is complete, the status (success/partial success) is updated to
+CarbonData store metadata. The table below lists the possible load status.</p>
+</li>
+</ul>
+<table>
+<thead>
+<tr>
+<th>Status</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>Success</td>
+<td>All the data is loaded into table and no bad records found.</td>
+</tr>
+<tr>
+<td>Partial Success</td>
+<td>Data is loaded into table and bad records are found. Bad records are stored at carbon.badrecords.location.</td>
+</tr>
+</tbody>
+</table>
+<p>In case of failure, the error will be logged in error log. Details of loads can be seen with <a href="dml-operation-on-carbondata.html#show-segments">SHOW SEGMENTS</a> command. The show segment command output consists of :</p>
+<ul>
+<li>SegmentSequenceId</li>
+<li>Status</li>
+<li>Load Start Time</li>
+<li>Load End Time</li>
+</ul>
+<p>The latest load will be displayed first in the output.</p>
+<p>Refer to <a href="dml-operation-on-carbondata.html">DML operations on CarbonData</a> for load commands.</p>
+<h2>
+<a id="deleting-data" class="anchor" href="#deleting-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deleting Data</h2>
+<ul>
+<li>
+<p><strong>Scenario</strong></p>
+<p>If you have loaded wrong data into the table, or too many bad records are present and you want to modify and reload the data, you can delete required data loads.
+The load can be deleted using the Segment Sequence Id or if the table contains date field then the data can be deleted using the date field.
+If there are some specific records that need to be deleted based on some filter condition(s) we can delete by records.</p>
+</li>
+<li>
+<p><strong>Procedure</strong></p>
+<p>The loaded data can be deleted in the following ways:</p>
+<ul>
+<li>
+<p>Delete by Segment ID</p>
+<p>After you get the segment ID of the segment that you want to delete, execute the delete command for the selected segment.
+The status of deleted segment is updated to Marked for delete / Marked for Update.</p>
+</li>
+</ul>
+</li>
+</ul>
+<table>
+<thead>
+<tr>
+<th>SegmentSequenceId</th>
+<th>Status</th>
+<th>Load Start Time</th>
+<th>Load End Time</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>0</td>
+<td>Success</td>
+<td>2015-11-19 19:14:...</td>
+<td>2015-11-19 19:14:...</td>
+</tr>
+<tr>
+<td>1</td>
+<td>Marked for Update</td>
+<td>2015-11-19 19:54:...</td>
+<td>2015-11-19 20:08:...</td>
+</tr>
+<tr>
+<td>2</td>
+<td>Marked for Delete</td>
+<td>2015-11-19 20:25:...</td>
+<td>2015-11-19 20:49:...</td>
+</tr>
+</tbody>
+</table>
+<ul>
+<li>
+<p>Delete by Date Field</p>
+<p>If the table contains date field, you can delete the data based on a specific date.</p>
+</li>
+<li>
+<p>Delete by Record</p>
+<p>To delete records from CarbonData table based on some filter Condition(s).</p>
+<p>For delete commands refer to <a href="dml-operation-on-carbondata.html">DML operations on CarbonData</a>.</p>
+</li>
+<li>
+<p><strong>NOTE</strong>:</p>
+<ul>
+<li>
+<p>When the delete segment DML is called, segment will not be deleted physically from the file system. Instead the segment status will be marked as "Marked for Delete". For the query execution, this deleted segment will be excluded.</p>
+</li>
+<li>
+<p>The deleted segment will be deleted physically during the next load operation and only after the maximum query execution time configured using "max.query.execution.time". By default it is 60 minutes.</p>
+</li>
+<li>
+<p>If the user wants to force delete the segment physically then he can use CLEAN FILES Command.</p>
+</li>
+</ul>
+</li>
+</ul>
+<p>Example :</p>
+<pre><code>CLEAN FILES FOR TABLE table1
+</code></pre>
+<p>This DML will physically delete the segment which are "Marked for delete" and "Compacted" immediately.</p>
+<h2>
+<a id="compacting-data" class="anchor" href="#compacting-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Compacting Data</h2>
+<ul>
+<li>
+<p><strong>Scenario</strong></p>
+<p>Frequent data ingestion results in several fragmented CarbonData files in the store directory. Since data is sorted only within each load, the indices perform only within each
+load. This means that there will be one index for each load and as number of data load increases, the number of indices also increases. As each index works only on one load,
+the performance of indices is reduced. CarbonData provides provision for compacting the loads. Compaction process combines several segments into one large segment by merge sorting the data from across the segments.</p>
+</li>
+<li>
+<p><strong>Procedure</strong></p>
+<p>There are two types of compaction Minor and Major compaction.</p>
+<ul>
+<li>
+<p><strong>Minor Compaction</strong></p>
+<p>In minor compaction the user can specify how many loads to be merged. Minor compaction triggers for every data load if the parameter carbon.enable.auto.load.merge is set. If any segments are available to be merged, then compaction will
+run parallel with data load. There are 2 levels in minor compaction.</p>
+<ul>
+<li>Level 1: Merging of the segments which are not yet compacted.</li>
+<li>Level 2: Merging of the compacted segments again to form a bigger segment.</li>
+</ul>
+</li>
+<li>
+<p><strong>Major Compaction</strong></p>
+<p>In Major compaction, many segments can be merged into one big segment. User will specify the compaction size until which segments can be merged. Major compaction is usually done during the off-peak time.</p>
+</li>
+</ul>
+<p>There are number of parameters related to Compaction that can be set in carbon.properties file</p>
+</li>
+</ul>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Default</th>
+<th>Application</th>
+<th>Description</th>
+<th>Valid Values</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>carbon.compaction.level.threshold</td>
+<td>4, 3</td>
+<td>Minor</td>
+<td>This property is for minor compaction which decides how many segments to be merged. Example: If it is set as "2, 3", then minor compaction will be triggered for every 2 segments in level 1. 3 is the number of level 1 compacted segment which is further compacted to new segment in level 2.</td>
+<td>NA</td>
+</tr>
+<tr>
+<td>carbon.major.compaction.size</td>
+<td>1024 MB</td>
+<td>Major</td>
+<td>Major compaction size can be configured using this parameter. Sum of the segments which is below this threshold will be merged.</td>
+<td>NA</td>
+</tr>
+<tr>
+<td>carbon.numberof.preserve.segments</td>
+<td>0</td>
+<td>Minor/Major</td>
+<td>This property configures number of segments to preserve from being compacted. Example: carbon.numberof.preserve.segments=2 then 2 latest segments will always be excluded from the compaction. No segments will be preserved by default.</td>
+<td>0-100</td>
+</tr>
+<tr>
+<td>carbon.allowed.compaction.days</td>
+<td>0</td>
+<td>Minor/Major</td>
+<td>Compaction will merge the segments which are loaded within the specific number of days configured. Example: If the configuration is 2, then the segments which are loaded in the time frame of 2 days only will get merged. Segments which are loaded 2 days apart will not be merged. This is disabled by default.</td>
+<td>0-100</td>
+</tr>
+<tr>
+<td>carbon.number.of.cores.while.compacting</td>
+<td>2</td>
+<td>Minor/Major</td>
+<td>Number of cores which is used to write data during compaction.</td>
+<td>0-100</td>
+</tr>
+</tbody>
+</table>
+<p>For compaction commands refer to <a href="ddl-operation-on-carbondata.html">DDL operations on CarbonData</a></p>
+<h2>
+<a id="updating-data" class="anchor" href="#updating-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Updating Data</h2>
+<ul>
+<li>
+<p><strong>Scenario</strong></p>
+<p>Sometimes after the data has been ingested into the System, it is required to be updated. Also there may be situations where some specific columns need to be updated
+on the basis of column expression and optional filter conditions.</p>
+</li>
+<li>
+<p><strong>Procedure</strong></p>
+<p>To update we need to specify the column expression with an optional filter condition(s).</p>
+<p>For update commands refer to <a href="dml-operation-on-carbondata.html">DML operations on CarbonData</a>.</p>
+</li>
+</ul>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/bee56334/content/ddl-operation-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/ddl-operation-on-carbondata.html b/content/ddl-operation-on-carbondata.html
new file mode 100644
index 0000000..444428f
--- /dev/null
+++ b/content/ddl-operation-on-carbondata.html
@@ -0,0 +1,748 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
+                                   target="_blank">Apache CarbonData 1.2.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
+                                   target="_blank">Apache CarbonData 1.1.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
+                                   target="_blank">Apache CarbonData 1.1.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
+                                   target="_blank">Apache CarbonData 1.0.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.2.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.1</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="row">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="ddl-operations-on-carbondata" class="anchor" href="#ddl-operations-on-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DDL Operations on CarbonData</h1>
+<p>This tutorial guides you through the data definition language support provided by CarbonData.</p>
+<h2>
+<a id="overview" class="anchor" href="#overview" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Overview</h2>
+<p>The following DDL operations are supported in CarbonData :</p>
+<ul>
+<li><a href="#create-table">CREATE TABLE</a></li>
+<li><a href="#show-table">SHOW TABLE</a></li>
+<li>
+<a href="#alter-table">ALTER TABLE</a>
+<ul>
+<li><a href="#rename-table">RENAME TABLE</a></li>
+<li><a href="#add-column">ADD COLUMN</a></li>
+<li><a href="#drop-columns">DROP COLUMNS</a></li>
+<li><a href="#change-data-type">CHANGE DATA TYPE</a></li>
+</ul>
+</li>
+<li><a href="#drop-table">DROP TABLE</a></li>
+<li><a href="#compaction">COMPACTION</a></li>
+<li><a href="#bucketing">BUCKETING</a></li>
+</ul>
+<h2>
+<a id="create-table" class="anchor" href="#create-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE</h2>
+<p>This command can be used to create a CarbonData table by specifying the list of fields along with the table properties.</p>
+<pre><code>   CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
+                    [(col_name data_type , ...)]
+   STORED BY 'carbondata'
+   [TBLPROPERTIES (property_name=property_value, ...)]
+   // All Carbon's additional table options will go into properties
+</code></pre>
+<h3>
+<a id="parameter-description" class="anchor" href="#parameter-description" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Optional</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>db_name</td>
+<td>Name of the database. Database name should consist of alphanumeric characters and underscore(_) special character.</td>
+<td>YES</td>
+</tr>
+<tr>
+<td>field_list</td>
+<td>Comma separated List of fields with data type. The field names should consist of alphanumeric characters and underscore(_) special character.</td>
+<td>NO</td>
+</tr>
+<tr>
+<td>table_name</td>
+<td>The name of the table in Database. Table name should consist of alphanumeric characters and underscore(_) special character.</td>
+<td>NO</td>
+</tr>
+<tr>
+<td>STORED BY</td>
+<td>"org.apache.carbondata.format", identifies and creates a CarbonData table.</td>
+<td>NO</td>
+</tr>
+<tr>
+<td>TBLPROPERTIES</td>
+<td>List of CarbonData table properties.</td>
+<td>YES</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="usage-guidelines" class="anchor" href="#usage-guidelines" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
+<p>Following are the guidelines for using table properties.</p>
+<ul>
+<li>
+<p><strong>Dictionary Encoding Configuration</strong></p>
+<p>Dictionary encoding is enabled by default for all String columns, and disabled for non-String columns. You can include and exclude columns for dictionary encoding.</p>
+</li>
+</ul>
+<pre><code>       TBLPROPERTIES ('DICTIONARY_EXCLUDE'='column1, column2')
+       TBLPROPERTIES ('DICTIONARY_INCLUDE'='column1, column2')
+</code></pre>
+<p>Here, DICTIONARY_EXCLUDE will exclude dictionary creation. This is applicable for high-cardinality columns and is an optional parameter. DICTIONARY_INCLUDE will generate dictionary for the columns specified in the list.</p>
+<ul>
+<li>
+<p><strong>Table Block Size Configuration</strong></p>
+<p>The block size of table files can be defined using the property TABLE_BLOCKSIZE. It accepts only integer values. The default value is 1024 MB and supports a range of 1 MB to 2048 MB.
+If you do not specify this value in the DDL command, default value is used.</p>
+</li>
+</ul>
+<pre><code>       TBLPROPERTIES ('TABLE_BLOCKSIZE'='512')
+</code></pre>
+<p>Here 512 MB means the block size of this table is 512 MB, you can also set it as 512M or 512.</p>
+<ul>
+<li>
+<p><strong>Inverted Index Configuration</strong></p>
+<p>Inverted index is very useful to improve compression ratio and query speed, especially for those low-cardinality columns which are in reward position.
+By default inverted index is enabled. The user can disable the inverted index creation for some columns.</p>
+</li>
+</ul>
+<pre><code>       TBLPROPERTIES ('NO_INVERTED_INDEX'='column1, column3')
+</code></pre>
+<p>No inverted index shall be generated for the columns specified in NO_INVERTED_INDEX. This property is applicable on columns with high-cardinality and is an optional parameter.</p>
+<p>NOTE:</p>
+<ul>
+<li>
+<p>By default all columns other than numeric datatype are treated as dimensions and all columns of numeric datatype are treated as measures.</p>
+</li>
+<li>
+<p>All dimensions except complex datatype columns are part of multi dimensional key(MDK). This behavior can be overridden by using TBLPROPERTIES. If the user wants to keep any column (except columns of complex datatype) in multi dimensional key then he can keep the columns either in DICTIONARY_EXCLUDE or DICTIONARY_INCLUDE.</p>
+</li>
+<li>
+<p><strong>Sort Columns Configuration</strong></p>
+<p>"SORT_COLUMN" property is for users to specify which columns belong to the MDK index. If user don't specify "SORT_COLUMN" property, by default MDK index be built by using all dimension columns except complex datatype column.</p>
+</li>
+</ul>
+<pre><code>       TBLPROPERTIES ('SORT_COLUMNS'='column1, column3')
+</code></pre>
+<h3>
+<a id="example" class="anchor" href="#example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
+<pre><code>    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_EXCLUDE'='storeCity',
+                     'DICTIONARY_INCLUDE'='productNumber',
+                     'NO_INVERTED_INDEX'='productBatch',
+                     'SORT_COLUMNS'='productName,storeCity')
+</code></pre>
+<ul>
+<li><strong>SORT_COLUMNS</strong></li>
+</ul>
+<pre><code>This table property specifies the order of the sort column.
+</code></pre>
+<pre><code>    TBLPROPERTIES('SORT_COLUMNS'='column1, column3')
+</code></pre>
+<p>NOTE:</p>
+<ul>
+<li>
+<p>If this property is not specified, then by default SORT_COLUMNS consist of all dimension (exclude Complex Column).</p>
+</li>
+<li>
+<p>If this property is specified but with empty argument, then the table will be loaded without sort. For example, ('SORT_COLUMNS'='')</p>
+</li>
+</ul>
+<h2>
+<a id="show-table" class="anchor" href="#show-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW TABLE</h2>
+<p>This command can be used to list all the tables in current database or all the tables of a specific database.</p>
+<pre><code>  SHOW TABLES [IN db_Name];
+</code></pre>
+<h3>
+<a id="parameter-description-1" class="anchor" href="#parameter-description-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Optional</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>IN db_Name</td>
+<td>Name of the database. Required only if tables of this specific database are to be listed.</td>
+<td>YES</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="example-1" class="anchor" href="#example-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
+<pre><code>  SHOW TABLES IN ProductSchema;
+</code></pre>
+<h2>
+<a id="alter-table" class="anchor" href="#alter-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>ALTER TABLE</h2>
+<p>The following section shall discuss the commands to modify the physical or logical state of the existing table(s).</p>
+<h3>
+<a id="rename-table" class="anchor" href="#rename-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><strong>RENAME TABLE</strong>
+</h3>
+<p>This command is used to rename the existing table.</p>
+<pre><code>    ALTER TABLE [db_name.]table_name RENAME TO new_table_name;
+</code></pre>
+<h4>
+<a id="parameter-description-2" class="anchor" href="#parameter-description-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h4>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Optional</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>db_Name</td>
+<td>Name of the database. If this parameter is left unspecified, the current database is selected.</td>
+<td>YES</td>
+</tr>
+<tr>
+<td>table_name</td>
+<td>Name of the existing table.</td>
+<td>NO</td>
+</tr>
+<tr>
+<td>new_table_name</td>
+<td>New table name for the existing table.</td>
+<td>NO</td>
+</tr>
+</tbody>
+</table>
+<h4>
+<a id="usage-guidelines-1" class="anchor" href="#usage-guidelines-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h4>
+<ul>
+<li>
+<p>Queries that require the formation of path using the table name for reading carbon store files, running in parallel with Rename command might fail during the renaming operation.</p>
+</li>
+<li>
+<p>Renaming of Secondary index table(s) is not permitted.</p>
+</li>
+</ul>
+<h4>
+<a id="examples" class="anchor" href="#examples" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples:</h4>
+<pre><code>    ALTER TABLE carbon RENAME TO carbondata;
+</code></pre>
+<pre><code>    ALTER TABLE test_db.carbon RENAME TO test_db.carbondata;
+</code></pre>
+<h3>
+<a id="add-column" class="anchor" href="#add-column" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><strong>ADD COLUMN</strong>
+</h3>
+<p>This command is used to add a new column to the existing table.</p>
+<pre><code>    ALTER TABLE [db_name.]table_name ADD COLUMNS (col_name data_type,...)
+    TBLPROPERTIES('DICTIONARY_INCLUDE'='col_name,...',
+    'DICTIONARY_EXCLUDE'='col_name,...',
+    'DEFAULT.VALUE.COLUMN_NAME'='default_value');
+</code></pre>
+<h4>
+<a id="parameter-description-3" class="anchor" href="#parameter-description-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h4>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Optional</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>db_Name</td>
+<td>Name of the database. If this parameter is left unspecified, the current database is selected.</td>
+<td>YES</td>
+</tr>
+<tr>
+<td>table_name</td>
+<td>Name of the existing table.</td>
+<td>NO</td>
+</tr>
+<tr>
+<td>col_name data_type</td>
+<td>Name of comma-separated column with data type. Column names contain letters, digits, and underscores (_).</td>
+<td>NO</td>
+</tr>
+</tbody>
+</table>
+<p>NOTE: Do not name the column after name, tupleId, PositionId, and PositionReference when creating Carbon tables because they are used internally by UPDATE, DELETE, and secondary index.</p>
+<h4>
+<a id="usage-guidelines-2" class="anchor" href="#usage-guidelines-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h4>
+<ul>
+<li>
+<p>Apart from DICTIONARY_INCLUDE, DICTIONARY_EXCLUDE and default_value no other property will be read. If any other property name is specified, error will not be thrown, it will be ignored.</p>
+</li>
+<li>
+<p>If default value is not specified, then NULL will be considered as the default value for the column.</p>
+</li>
+<li>
+<p>For addition of column, if DICTIONARY_INCLUDE and DICTIONARY_EXCLUDE are not specified, then the decision will be taken based on data type of the column.</p>
+</li>
+</ul>
+<h4>
+<a id="examples-1" class="anchor" href="#examples-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples:</h4>
+<pre><code>    ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING);
+</code></pre>
+<pre><code>    ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING)
+    TBLPROPERTIES('DICTIONARY_EXCLUDE'='b1');
+</code></pre>
+<pre><code>    ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING)
+    TBLPROPERTIES('DICTIONARY_INCLUDE'='a1');
+</code></pre>
+<pre><code>    ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING)
+    TBLPROPERTIES('DEFAULT.VALUE.a1'='10');
+</code></pre>
+<h3>
+<a id="drop-columns" class="anchor" href="#drop-columns" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><strong>DROP COLUMNS</strong>
+</h3>
+<p>This command is used to delete a existing column or multiple columns in a table.</p>
+<pre><code>    ALTER TABLE [db_name.]table_name DROP COLUMNS (col_name, ...);
+</code></pre>
+<h4>
+<a id="parameter-description-4" class="anchor" href="#parameter-description-4" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h4>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Optional</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>db_Name</td>
+<td>Name of the database. If this parameter is left unspecified, the current database is selected.</td>
+<td>YES</td>
+</tr>
+<tr>
+<td>table_name</td>
+<td>Name of the existing table.</td>
+<td>NO</td>
+</tr>
+<tr>
+<td>col_name</td>
+<td>Name of comma-separated column with data type. Column names contain letters, digits, and underscores (_)</td>
+<td>NO</td>
+</tr>
+</tbody>
+</table>
+<h4>
+<a id="usage-guidelines-3" class="anchor" href="#usage-guidelines-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h4>
+<ul>
+<li>
+<p>Deleting a column will also clear the dictionary files, provided the column is of type dictionary.</p>
+</li>
+<li>
+<p>For delete column operation, there should be at least one key column that exists in the schema after deletion else error message will be displayed and the operation shall fail.</p>
+</li>
+</ul>
+<h4>
+<a id="examples-2" class="anchor" href="#examples-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples:</h4>
+<p>If the table contains 4 columns namely a1, b1, c1, and d1.</p>
+<ul>
+<li><strong>To delete a single column:</strong></li>
+</ul>
+<pre><code>   ALTER TABLE carbon DROP COLUMNS (b1);
+</code></pre>
+<pre><code>    ALTER TABLE test_db.carbon DROP COLUMNS (b1);
+</code></pre>
+<ul>
+<li><strong>To delete multiple columns:</strong></li>
+</ul>
+<pre><code>   ALTER TABLE carbon DROP COLUMNS (c1,d1);
+</code></pre>
+<h3>
+<a id="change-data-type" class="anchor" href="#change-data-type" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><strong>CHANGE DATA TYPE</strong>
+</h3>
+<p>This command is used to change the data type from INT to BIGINT or decimal precision from lower to higher.</p>
+<pre><code>    ALTER TABLE [db_name.]table_name
+    CHANGE col_name col_name changed_column_type;
+</code></pre>
+<h4>
+<a id="parameter-description-5" class="anchor" href="#parameter-description-5" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h4>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Optional</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>db_Name</td>
+<td>Name of the database. If this parameter is left unspecified, the current database is selected.</td>
+<td>YES</td>
+</tr>
+<tr>
+<td>table_name</td>
+<td>Name of the existing table.</td>
+<td>NO</td>
+</tr>
+<tr>
+<td>col_name</td>
+<td>Name of comma-separated column with data type. Column names contain letters, digits, and underscores (_).</td>
+<td>NO</td>
+</tr>
+<tr>
+<td>changed_column_type</td>
+<td>The change in the data type.</td>
+<td>NO</td>
+</tr>
+</tbody>
+</table>
+<h4>
+<a id="usage-guidelines-4" class="anchor" href="#usage-guidelines-4" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h4>
+<ul>
+<li>Change of decimal data type from lower precision to higher precision will only be supported for cases where there is no data loss.</li>
+</ul>
+<h4>
+<a id="valid-scenarios" class="anchor" href="#valid-scenarios" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Valid Scenarios</h4>
+<ul>
+<li>
+<p>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.</p>
+</li>
+<li>
+<p>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.</p>
+</li>
+<li>
+<p>Note :The allowed range is 38,38 (precision, scale) and is a valid upper case scenario which is not resulting in data loss.</p>
+</li>
+</ul>
+<h4>
+<a id="examples-3" class="anchor" href="#examples-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples:</h4>
+<ul>
+<li><strong>Changing data type of column a1 from INT to BIGINT</strong></li>
+</ul>
+<pre><code>   ALTER TABLE test_db.carbon CHANGE a1 a1 BIGINT;
+</code></pre>
+<ul>
+<li><strong>Changing decimal precision of column a1 from 10 to 18.</strong></li>
+</ul>
+<pre><code>   ALTER TABLE test_db.carbon CHANGE a1 a1 DECIMAL(18,2);
+</code></pre>
+<h2>
+<a id="drop-table" class="anchor" href="#drop-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DROP TABLE</h2>
+<p>This command is used to delete an existing table.</p>
+<pre><code>  DROP TABLE [IF EXISTS] [db_name.]table_name;
+</code></pre>
+<h3>
+<a id="parameter-description-6" class="anchor" href="#parameter-description-6" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Optional</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>db_Name</td>
+<td>Name of the database. If not specified, current database will be selected.</td>
+<td>YES</td>
+</tr>
+<tr>
+<td>table_name</td>
+<td>Name of the table to be deleted.</td>
+<td>NO</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="example-2" class="anchor" href="#example-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
+<pre><code>  DROP TABLE IF EXISTS productSchema.productSalesTable;
+</code></pre>
+<h2>
+<a id="compaction" class="anchor" href="#compaction" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COMPACTION</h2>
+<p>This command merges the specified number of segments into one segment. This enhances the query performance of the table.</p>
+<pre><code>  ALTER TABLE [db_name.]table_name COMPACT 'MINOR/MAJOR';
+</code></pre>
+<p>To get details about Compaction refer to <a href="data-management.html">Data Management</a></p>
+<h3>
+<a id="parameter-description-7" class="anchor" href="#parameter-description-7" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Optional</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>db_name</td>
+<td>Database name, if it is not specified then it uses current database.</td>
+<td>YES</td>
+</tr>
+<tr>
+<td>table_name</td>
+<td>The name of the table in provided database.</td>
+<td>NO</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="syntax" class="anchor" href="#syntax" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Syntax</h3>
+<ul>
+<li><strong>Minor Compaction</strong></li>
+</ul>
+<pre><code>ALTER TABLE table_name COMPACT 'MINOR';
+</code></pre>
+<ul>
+<li><strong>Major Compaction</strong></li>
+</ul>
+<pre><code>ALTER TABLE table_name COMPACT 'MAJOR';
+</code></pre>
+<h2>
+<a id="bucketing" class="anchor" href="#bucketing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>BUCKETING</h2>
+<p>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, a 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.</p>
+<pre><code>   CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
+                    [(col_name data_type, ...)]
+   STORED BY 'carbondata'
+   TBLPROPERTIES('BUCKETNUMBER'='noOfBuckets',
+   'BUCKETCOLUMNS'='columnname')
+</code></pre>
+<h3>
+<a id="parameter-description-8" class="anchor" href="#parameter-description-8" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Optional</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>BUCKETNUMBER</td>
+<td>Specifies the number of Buckets to be created.</td>
+<td>No</td>
+</tr>
+<tr>
+<td>BUCKETCOLUMNS</td>
+<td>Specify the columns to be considered for Bucketing</td>
+<td>No</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="usage-guidelines-5" class="anchor" href="#usage-guidelines-5" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
+<ul>
+<li>
+<p>The feature is supported for Spark 1.6.2 onwards, but the performance optimization is evident from Spark 2.1 onwards.</p>
+</li>
+<li>
+<p>Bucketing can not be performed for columns of Complex Data Types.</p>
+</li>
+<li>
+<p>Columns in the BUCKETCOLUMN parameter must be only dimension. The BUCKETCOLUMN parameter can not be a measure or a combination of measures and dimensions.</p>
+</li>
+</ul>
+<h3>
+<a id="example-3" class="anchor" href="#example-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
+<pre><code> 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 ('DICTIONARY_EXCLUDE'='productName',
+                  'DICTIONARY_INCLUDE'='productNumber,saleQuantity',
+                  'NO_INVERTED_INDEX'='productBatch',
+                  'BUCKETNUMBER'='4',
+                  'BUCKETCOLUMNS'='productName')
+</code></pre>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/bee56334/content/dml-operation-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/dml-operation-on-carbondata.html b/content/dml-operation-on-carbondata.html
new file mode 100644
index 0000000..b6a5642
--- /dev/null
+++ b/content/dml-operation-on-carbondata.html
@@ -0,0 +1,716 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
+                                   target="_blank">Apache CarbonData 1.2.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
+                                   target="_blank">Apache CarbonData 1.1.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
+                                   target="_blank">Apache CarbonData 1.1.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
+                                   target="_blank">Apache CarbonData 1.0.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.2.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.1</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="row">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="dml-operations-on-carbondata" class="anchor" href="#dml-operations-on-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DML Operations on CarbonData</h1>
+<p>This tutorial guides you through the data manipulation language support provided by CarbonData.</p>
+<h2>
+<a id="overview" class="anchor" href="#overview" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Overview</h2>
+<p>The following DML operations are supported in CarbonData :</p>
+<ul>
+<li><a href="#load-data">LOAD DATA</a></li>
+<li><a href="#insert-data-into-a-carbondata-table">INSERT DATA INTO A CARBONDATA TABLE</a></li>
+<li><a href="#show-segments">SHOW SEGMENTS</a></li>
+<li><a href="#delete-segment-by-id">DELETE SEGMENT BY ID</a></li>
+<li><a href="#delete-segment-by-date">DELETE SEGMENT BY DATE</a></li>
+<li><a href="#update-carbondata-table">UPDATE CARBONDATA TABLE</a></li>
+<li><a href="#delete-records-from-carbondata-table">DELETE RECORDS FROM CARBONDATA TABLE</a></li>
+</ul>
+<h2>
+<a id="load-data" class="anchor" href="#load-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LOAD DATA</h2>
+<p>This command loads the user data in raw format to the CarbonData specific data format store, this allows CarbonData to provide good performance while querying the data.
+Please visit <a href="data-management.html">Data Management</a> for more details on LOAD.</p>
+<h3>
+<a id="syntax" class="anchor" href="#syntax" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Syntax</h3>
+<pre><code>LOAD DATA [LOCAL] INPATH 'folder_path' 
+INTO TABLE [db_name.]table_name 
+OPTIONS(property_name=property_value, ...)
+</code></pre>
+<p>OPTIONS are not mandatory for data loading process. Inside OPTIONS user can provide either of any options like DELIMITER, QUOTECHAR, ESCAPECHAR, MULTILINE as per requirement.</p>
+<p>NOTE: The path shall be canonical path.</p>
+<h3>
+<a id="parameter-description" class="anchor" href="#parameter-description" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Optional</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>folder_path</td>
+<td>Path of raw csv data folder or file.</td>
+<td>NO</td>
+</tr>
+<tr>
+<td>db_name</td>
+<td>Database name, if it is not specified then it uses the current database.</td>
+<td>YES</td>
+</tr>
+<tr>
+<td>table_name</td>
+<td>The name of the table in provided database.</td>
+<td>NO</td>
+</tr>
+<tr>
+<td>OPTIONS</td>
+<td>Extra options provided to Load</td>
+<td>YES</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="usage-guidelines" class="anchor" href="#usage-guidelines" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
+<p>You can use the following options to load data:</p>
+<ul>
+<li>
+<p><strong>DELIMITER:</strong> Delimiters can be provided in the load command.</p>
+<pre><code>OPTIONS('DELIMITER'=',')
+</code></pre>
+</li>
+<li>
+<p><strong>QUOTECHAR:</strong> Quote Characters can be provided in the load command.</p>
+<pre><code>OPTIONS('QUOTECHAR'='"')
+</code></pre>
+</li>
+<li>
+<p><strong>COMMENTCHAR:</strong> Comment Characters can be provided in the load command if user want to comment lines.</p>
+<pre><code>OPTIONS('COMMENTCHAR'='#')
+</code></pre>
+</li>
+<li>
+<p><strong>FILEHEADER:</strong> Headers can be provided in the LOAD DATA command if headers are missing in the source files.</p>
+<pre><code>OPTIONS('FILEHEADER'='column1,column2') 
+</code></pre>
+</li>
+<li>
+<p><strong>MULTILINE:</strong> CSV with new line character in quotes.</p>
+<pre><code>OPTIONS('MULTILINE'='true') 
+</code></pre>
+</li>
+<li>
+<p><strong>ESCAPECHAR:</strong> Escape char can be provided if user want strict validation of escape character on CSV.</p>
+<pre><code>OPTIONS('ESCAPECHAR'='\') 
+</code></pre>
+</li>
+<li>
+<p><strong>COMPLEX_DELIMITER_LEVEL_1:</strong> Split the complex type data column in a row (eg., a$b$c --&gt; Array = {a,b,c}).</p>
+<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_1'='$') 
+</code></pre>
+</li>
+<li>
+<p><strong>COMPLEX_DELIMITER_LEVEL_2:</strong> Split the complex type nested data column in a row. Applies level_1 delimiter &amp; applies level_2 based on complex data type (eg., a:b$c:d --&gt; Array&gt; = {{a,b},{c,d}}).</p>
+<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_2'=':')
+</code></pre>
+</li>
+<li>
+<p><strong>ALL_DICTIONARY_PATH:</strong> All dictionary files path.</p>
+<pre><code>OPTIONS('ALL_DICTIONARY_PATH'='/opt/alldictionary/data.dictionary')
+</code></pre>
+</li>
+<li>
+<p><strong>COLUMNDICT:</strong> Dictionary file path for specified column.</p>
+<pre><code>OPTIONS('COLUMNDICT'='column1:dictionaryFilePath1,
+column2:dictionaryFilePath2')
+</code></pre>
+<p>NOTE: ALL_DICTIONARY_PATH and COLUMNDICT can't be used together.</p>
+</li>
+<li>
+<p><strong>DATEFORMAT:</strong> Date format for specified column.</p>
+<pre><code>OPTIONS('DATEFORMAT'='column1:dateFormat1, column2:dateFormat2')
+</code></pre>
+<p>NOTE: Date formats are specified by date pattern strings. The date pattern letters in CarbonData are same as in JAVA. Refer to <a href="http://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html" target=_blank>SimpleDateFormat</a>.</p>
+</li>
+<li>
+<p><strong>SINGLE_PASS:</strong> 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.</p>
+<p>This option specifies whether to use single pass for loading data or not. By default this option is set to FALSE.</p>
+<pre><code>OPTIONS('SINGLE_PASS'='TRUE')
+</code></pre>
+<p>Note :</p>
+<ul>
+<li>
+<p>If this option is set to TRUE then data loading will take less time.</p>
+</li>
+<li>
+<p>If this option is set to some invalid value other than TRUE or FALSE then it uses the default value.</p>
+</li>
+<li>
+<p>If this option is set to TRUE, then high.cardinality.identify.enable property will be disabled during data load.</p>
+</li>
+</ul>
+<h3>
+<a id="example" class="anchor" href="#example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
+</li>
+</ul>
+<pre><code>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'
+)
+</code></pre>
+<ul>
+<li>
+<p><strong>BAD RECORDS HANDLING:</strong> Methods of handling bad records are as follows:</p>
+<ul>
+<li>
+<p>Load all of the data before dealing with the errors.</p>
+</li>
+<li>
+<p>Clean or delete bad records before loading data or stop the loading when bad records are found.</p>
+</li>
+</ul>
+<pre><code>OPTIONS('BAD_RECORDS_LOGGER_ENABLE'='true', 'BAD_RECORD_PATH'='hdfs://hacluster/tmp/carbon', 'BAD_RECORDS_ACTION'='REDIRECT', 'IS_EMPTY_DATA_BAD_RECORD'='false')
+</code></pre>
+<p>NOTE:</p>
+<ul>
+<li>
+<p>If the REDIRECT option is used, Carbon 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.</p>
+</li>
+<li>
+<p>In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails.</p>
+</li>
+<li>
+<p>The maximum number of characters per column is 100000. If there are more than 100000 characters in a column, data loading will fail.</p>
+</li>
+</ul>
+</li>
+</ul>
+<h3>
+<a id="example-1" class="anchor" href="#example-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
+<pre><code>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');
+</code></pre>
+<p><strong>Bad Records Management Options:</strong></p>
+<table>
+<thead>
+<tr>
+<th>Options</th>
+<th>Default Value</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>BAD_RECORDS_LOGGER_ENABLE</td>
+<td>false</td>
+<td>Whether to create logs with details about bad records.</td>
+</tr>
+<tr>
+<td>BAD_RECORDS_ACTION</td>
+<td>FAIL</td>
+<td>Following are the four types of action for bad records:  FORCE: Auto-corrects the data by storing the bad records as NULL.  REDIRECT: Bad records are written to the raw CSV instead of being loaded.  IGNORE: Bad records are neither loaded nor written to the raw CSV.  FAIL: Data loading fails if any bad records are found.  NOTE: In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails.</td>
+</tr>
+<tr>
+<td>IS_EMPTY_DATA_BAD_RECORD</td>
+<td>false</td>
+<td>If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa.</td>
+</tr>
+<tr>
+<td>BAD_RECORD_PATH</td>
+<td>-</td>
+<td>Specifies the HDFS path where bad records are stored. By default the value is Null. This path must to be configured by the user if bad record logger is enabled or bad record action redirect.</td>
+</tr>
+</tbody>
+</table>
+<h2>
+<a id="insert-data-into-a-carbondata-table" class="anchor" href="#insert-data-into-a-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>INSERT DATA INTO A CARBONDATA TABLE</h2>
+<p>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.</p>
+<p><strong>NOTE</strong> :  The client node where the INSERT command is executing, must be part of the cluster.</p>
+<h3>
+<a id="syntax-1" class="anchor" href="#syntax-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Syntax</h3>
+<pre><code>INSERT INTO TABLE &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
+[ WHERE { &lt;filter_condition&gt; } ];
+</code></pre>
+<p>You can also omit the <code>table</code> keyword and write your query as:</p>
+<pre><code>INSERT INTO &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
+[ WHERE { &lt;filter_condition&gt; } ];
+</code></pre>
+<h3>
+<a id="parameter-description-1" class="anchor" href="#parameter-description-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>CARBON TABLE</td>
+<td>The name of the Carbon table in which you want to perform the insert operation.</td>
+</tr>
+<tr>
+<td>sourceTableName</td>
+<td>The table from which the records are read and inserted into destination CarbonData table.</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="usage-guidelines-1" class="anchor" href="#usage-guidelines-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
+<p>The following condition must be met for successful insert operation :</p>
+<ul>
+<li>The source table and the CarbonData table must have the same table schema.</li>
+<li>The table must be created.</li>
+<li>Overwrite is not supported for CarbonData table.</li>
+<li>The data type of source and destination table columns should be same, else the data from source table will be treated as bad records and the INSERT command fails.</li>
+<li>INSERT INTO command does not support partial success if bad records are found, it will fail.</li>
+<li>Data cannot be loaded or updated in source table while insert from source table to target table is in progress.</li>
+</ul>
+<p>To enable data load or update during insert operation, configure the following property to true.</p>
+<pre><code>carbon.insert.persist.enable=true
+</code></pre>
+<p>By default the above configuration will be false.</p>
+<p><strong>NOTE</strong>: Enabling this property will reduce the performance.</p>
+<h3>
+<a id="examples" class="anchor" href="#examples" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples</h3>
+<pre><code>INSERT INTO table1 SELECT item1 ,sum(item2 + 1000) as result FROM 
+table2 group by item1;
+</code></pre>
+<pre><code>INSERT INTO table1 SELECT item1, item2, item3 FROM table2 
+where item2='xyz';
+</code></pre>
+<pre><code>INSERT INTO table1 SELECT * FROM table2 
+where exists (select * from table3 
+where table2.item1 = table3.item1);
+</code></pre>
+<p><strong>The Status Success/Failure shall be captured in the driver log.</strong></p>
+<h2>
+<a id="show-segments" class="anchor" href="#show-segments" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW SEGMENTS</h2>
+<p>This command is used to get the segments of CarbonData table.</p>
+<pre><code>SHOW SEGMENTS FOR TABLE [db_name.]table_name 
+LIMIT number_of_segments;
+</code></pre>
+<h3>
+<a id="parameter-description-2" class="anchor" href="#parameter-description-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Optional</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>db_name</td>
+<td>Database name, if it is not specified then it uses the current database.</td>
+<td>YES</td>
+</tr>
+<tr>
+<td>table_name</td>
+<td>The name of the table in provided database.</td>
+<td>NO</td>
+</tr>
+<tr>
+<td>number_of_segments</td>
+<td>Limit the output to this number.</td>
+<td>YES</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="example-2" class="anchor" href="#example-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
+<pre><code>SHOW SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4;
+</code></pre>
+<h2>
+<a id="delete-segment-by-id" class="anchor" href="#delete-segment-by-id" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY ID</h2>
+<p>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.</p>
+<p>The following command will get the segmentID.</p>
+<pre><code>SHOW SEGMENTS FOR Table [db_name.]table_name LIMIT number_of_segments
+</code></pre>
+<p>After you retrieve the segment ID of the segment that you want to delete, execute the following command to delete the selected segment.</p>
+<pre><code>DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.ID IN (segment_id1, segments_id2, ...)
+</code></pre>
+<h3>
+<a id="parameter-description-3" class="anchor" href="#parameter-description-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Optional</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>segment_id</td>
+<td>Segment Id of the load.</td>
+<td>NO</td>
+</tr>
+<tr>
+<td>db_name</td>
+<td>Database name, if it is not specified then it uses the current database.</td>
+<td>YES</td>
+</tr>
+<tr>
+<td>table_name</td>
+<td>The name of the table in provided database.</td>
+<td>NO</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="example-3" class="anchor" href="#example-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
+<pre><code>DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0);
+DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0,5,8);
+</code></pre>
+<p>NOTE: Here 0.1 is compacted segment sequence id.</p>
+<h2>
+<a id="delete-segment-by-date" class="anchor" href="#delete-segment-by-date" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY DATE</h2>
+<p>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.</p>
+<pre><code>DELETE FROM TABLE [db_name.]table_name 
+WHERE SEGMENT.STARTTIME BEFORE DATE_VALUE
+</code></pre>
+<h3>
+<a id="parameter-description-4" class="anchor" href="#parameter-description-4" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Optional</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>DATE_VALUE</td>
+<td>Valid segment load start time value. All the segments before this specified date will be deleted.</td>
+<td>NO</td>
+</tr>
+<tr>
+<td>db_name</td>
+<td>Database name, if it is not specified then it uses the current database.</td>
+<td>YES</td>
+</tr>
+<tr>
+<td>table_name</td>
+<td>The name of the table in provided database.</td>
+<td>NO</td>
+</tr>
+</tbody>
+</table>
+<h3>
+<a id="example-4" class="anchor" href="#example-4" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
+<pre><code> DELETE FROM TABLE CarbonDatabase.CarbonTable 
+ WHERE SEGMENT.STARTTIME BEFORE '2017-06-01 12:05:06';  
+</code></pre>
+<h2>
+<a id="update-carbondata-table" class="anchor" href="#update-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Update CarbonData Table</h2>
+<p>This command will allow to update the carbon table based on the column expression and optional filter conditions.</p>
+<h3>
+<a id="syntax-2" class="anchor" href="#syntax-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Syntax</h3>
+<pre><code> UPDATE &lt;table_name&gt;
+ SET (column_name1, column_name2, ... column_name n) =
+ (column1_expression , column2_expression, ... column n_expression )
+ [ WHERE { &lt;filter_condition&gt; } ];
+</code></pre>
+<p>alternatively the following the command can also be used for updating the CarbonData Table :</p>
+<pre><code>UPDATE &lt;table_name&gt;
+SET (column_name1, column_name2) =
+(select sourceColumn1, sourceColumn2 from sourceTable
+[ WHERE { &lt;filter_condition&gt; } ] )
+[ WHERE { &lt;filter_condition&gt; } ];
+</code></pre>
+<h3>
+<a id="parameter-description-5" class="anchor" href="#parameter-description-5" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>table_name</td>
+<td>The name of the Carbon table in which you want to perform the update operation.</td>
+</tr>
+<tr>
+<td>column_name</td>
+<td>The destination columns to be updated.</td>
+</tr>
+<tr>
+<td>sourceColumn</td>
+<td>The source table column values to be updated in destination table.</td>
+</tr>
+<tr>
+<td>sourceTable</td>
+<td>The table from which the records are updated into destination Carbon table.</td>
+</tr>
+</tbody>
+</table>
+<p>NOTE: This functionality is currently not supported in Spark 2.x and will support soon.</p>
+<h3>
+<a id="usage-guidelines-2" class="anchor" href="#usage-guidelines-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
+<p>The following conditions must be met for successful updation :</p>
+<ul>
+<li>The update command fails if multiple input rows in source table are matched with single row in destination table.</li>
+<li>If the source table generates empty records, the update operation will complete successfully without updating the table.</li>
+<li>If a source table row does not correspond to any of the existing rows in a destination table, the update operation will complete successfully without updating the table.</li>
+<li>In sub-query, if the source table and the target table are same, then the update operation fails.</li>
+<li>If the sub-query used in UPDATE statement contains aggregate method or group by query, then the UPDATE operation fails.</li>
+</ul>
+<h3>
+<a id="examples-1" class="anchor" href="#examples-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples</h3>
+<p>Update is not supported for queries that contain aggregate or group by.</p>
+<pre><code> UPDATE t_carbn01 a
+ SET (a.item_type_code, a.profit) = ( SELECT b.item_type_cd,
+ sum(b.profit) from t_carbn01b b
+ WHERE item_type_cd =2 group by item_type_code);
+</code></pre>
+<p>Here the Update Operation fails as the query contains aggregate function sum(b.profit) and group by clause in the sub-query.</p>
+<pre><code>UPDATE carbonTable1 d
+SET(d.column3,d.column5 ) = (SELECT s.c33 ,s.c55
+FROM sourceTable1 s WHERE d.column1 = s.c11)
+WHERE d.column1 = 'china' EXISTS( SELECT * from table3 o where o.c2 &gt; 1);
+</code></pre>
+<pre><code>UPDATE carbonTable1 d SET (c3) = (SELECT s.c33 from sourceTable1 s
+WHERE d.column1 = s.c11)
+WHERE exists( select * from iud.other o where o.c2 &gt; 1);
+</code></pre>
+<pre><code>UPDATE carbonTable1 SET (c2, c5 ) = (c2 + 1, concat(c5 , "y" ));
+</code></pre>
+<pre><code>UPDATE carbonTable1 d SET (c2, c5 ) = (c2 + 1, "xyx")
+WHERE d.column1 = 'india';
+</code></pre>
+<pre><code>UPDATE carbonTable1 d SET (c2, c5 ) = (c2 + 1, "xyx")
+WHERE d.column1 = 'india'
+and EXISTS( SELECT * FROM table3 o WHERE o.column2 &gt; 1);
+</code></pre>
+<p><strong>The Status Success/Failure shall be captured in the driver log and the client.</strong></p>
+<h2>
+<a id="delete-records-from-carbondata-table" class="anchor" href="#delete-records-from-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Delete Records from CarbonData Table</h2>
+<p>This command allows us to delete records from CarbonData table.</p>
+<h3>
+<a id="syntax-3" class="anchor" href="#syntax-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Syntax</h3>
+<pre><code>DELETE FROM table_name [WHERE expression];
+</code></pre>
+<h3>
+<a id="parameter-description-6" class="anchor" href="#parameter-description-6" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>table_name</td>
+<td>The name of the Carbon table in which you want to perform the delete.</td>
+</tr>
+</tbody>
+</table>
+<p>NOTE: This functionality is currently not supported in Spark 2.x and will support soon.</p>
+<h3>
+<a id="examples-2" class="anchor" href="#examples-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples</h3>
+<pre><code>DELETE FROM columncarbonTable1 d WHERE d.column1  = 'china';
+</code></pre>
+<pre><code>DELETE FROM dest WHERE column1 IN ('china', 'USA');
+</code></pre>
+<pre><code>DELETE FROM columncarbonTable1
+WHERE column1 IN (SELECT column11 FROM sourceTable2);
+</code></pre>
+<pre><code>DELETE FROM columncarbonTable1
+WHERE column1 IN (SELECT column11 FROM sourceTable2 WHERE
+column1 = 'USA');
+</code></pre>
+<pre><code>DELETE FROM columncarbonTable1 WHERE column2 &gt;= 4;
+</code></pre>
+<p><strong>The Status Success/Failure shall be captured in the driver log and the client.</strong></p>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file


[06/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/supported-data-types-in-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/supported-data-types-in-carbondata.html b/src/main/webapp/supported-data-types-in-carbondata.html
index 780cab0..e0ad647 100644
--- a/src/main/webapp/supported-data-types-in-carbondata.html
+++ b/src/main/webapp/supported-data-types-in-carbondata.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -189,7 +240,7 @@
 <li>VARCHAR</li>
 </ul>
 <p><strong>NOTE</strong>: For string longer than 32000 characters, use <code>LONG_STRING_COLUMNS</code> in table property.
-Please refer to TBLProperties in <a href="https://github.com/apache/carbondata/blob/master/docs/data-management-on-carbondata.html#create-table">CreateTable</a> for more information.</p>
+Please refer to TBLProperties in <a href="./ddl-of-carbondata.html#create-table">CreateTable</a> for more information.</p>
 </li>
 <li>
 <p>Complex Types</p>
@@ -208,6 +259,17 @@ Please refer to TBLProperties in <a href="https://github.com/apache/carbondata/b
 </ul>
 </li>
 </ul>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -223,4 +285,4 @@ Please refer to TBLProperties in <a href="https://github.com/apache/carbondata/b
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/timeseries-datamap-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/timeseries-datamap-guide.html b/src/main/webapp/timeseries-datamap-guide.html
index 548be2e..497b02f 100644
--- a/src/main/webapp/timeseries-datamap-guide.html
+++ b/src/main/webapp/timeseries-datamap-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -168,16 +219,16 @@
 </ul>
 <h2>
 <a id="timeseries-datamap-introduction-alpha-feature" class="anchor" href="#timeseries-datamap-introduction-alpha-feature" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Timeseries DataMap Introduction (Alpha Feature)</h2>
-<p>Timeseries DataMap a pre-aggregate table implementation based on 'pre-aggregate' DataMap.
+<p>Timeseries DataMap is a pre-aggregate table implementation based on 'pre-aggregate' DataMap.
 Difference is that Timeseries DataMap has built-in understanding of time hierarchy and
 levels: year, month, day, hour, minute, so that it supports automatic roll-up in time dimension
 for query.</p>
+<p><strong>CAUTION:</strong> Current version of CarbonData does not support roll-up.It will be implemented in future versions.</p>
 <p>The data loading, querying, compaction command and its behavior is the same as preaggregate DataMap.
-Please refer to <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/preaggregate-datamap-guide.html" target=_blank>Pre-aggregate DataMap</a>
+Please refer to <a href="./preaggregate-datamap-guide.html">Pre-aggregate DataMap</a>
 for more information.</p>
 <p>To use this datamap, user can create multiple timeseries datamap on the main table which has
-a <em>event_time</em> column, one datamap for one time granularity. Then Carbondata can do automatic
-roll-up for queries on the main table.</p>
+a <em>event_time</em> column, one datamap for one time granularity.</p>
 <p>For example, below statement effectively create multiple pre-aggregate tables  on main table called
 <strong>timeseries</strong></p>
 <pre><code>CREATE DATAMAP agg_year
@@ -229,26 +280,15 @@ DMPROPERTIES (
 ) 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_granularity'='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
 </code></pre>
-<p>For querying timeseries data, Carbondata has builtin support for following time related UDF
-to enable automatically roll-up to the desired aggregation level</p>
+<p>For querying timeseries data, Carbondata has builtin support for following time related UDF</p>
 <pre><code>timeseries(timeseries column name, 'aggregation level')
 </code></pre>
 <pre><code>SELECT timeseries(order_time, 'hour'), sum(quantity) FROM sales GROUP BY timeseries(order_time,
 'hour')
 </code></pre>
 <p>It is <strong>not necessary</strong> to create pre-aggregate tables for each granularity unless required for
-query. Carbondata can roll-up the data and fetch it.</p>
+query.</p>
 <p>For Example: For main table <strong>sales</strong> , if following timeseries datamaps were created for day
 level and hour level pre-aggregate</p>
 <pre><code>  CREATE DATAMAP agg_day
@@ -271,7 +311,7 @@ level and hour level pre-aggregate</p>
   SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
    avg(price) FROM sales GROUP BY order_time, country, sex
 </code></pre>
-<p>Queries like below will be rolled-up and hit the timeseries datamaps</p>
+<p>Queries like below will not be rolled-up and hit the main table</p>
 <pre><code>Select timeseries(order_time, 'month'), sum(quantity) from sales group by timeseries(order_time,
   'month')
   
@@ -287,12 +327,23 @@ the future CarbonData release.</li>
 </ul>
 <h2>
 <a id="compacting-timeseries-datamp" class="anchor" href="#compacting-timeseries-datamp" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Compacting timeseries datamp</h2>
-<p>Refer to Compaction section in <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/preaggregate-datamap-guide.html" target=_blank>preaggregation datamap</a>.
+<p>Refer to Compaction section in <a href="./preaggregate-datamap-guide.html">preaggregation datamap</a>.
 Same applies to timeseries datamap.</p>
 <h2>
 <a id="data-management-on-timeseries-datamap" class="anchor" href="#data-management-on-timeseries-datamap" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Management on timeseries datamap</h2>
-<p>Refer to Data Management section in <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/preaggregate-datamap-guide.html" target=_blank>preaggregation datamap</a>.
+<p>Refer to Data Management section in <a href="./preaggregate-datamap-guide.html">preaggregation datamap</a>.
 Same applies to timeseries datamap.</p>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -308,4 +359,4 @@ Same applies to timeseries datamap.</p>
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/troubleshooting.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/troubleshooting.html b/src/main/webapp/troubleshooting.html
index c668dc9..7e0cd60 100644
--- a/src/main/webapp/troubleshooting.html
+++ b/src/main/webapp/troubleshooting.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,59 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./videogallery.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__trouble nav__item" href="./troubleshooting.html">Troubleshooting</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./videogallery.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__trouble navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -348,6 +401,10 @@ For example, you can use scp to copy this file to all the nodes.</p>
 <p>Behaviour not supported.</p>
 <p><strong>Procedure</strong></p>
 <p>A single column that can be considered as dimension is mandatory for table creation.</p>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__trouble').addClass('selected'); });
+</script>
 </div>
 </div>
 </div>
@@ -363,4 +420,4 @@ For example, you can use scp to copy this file to all the nodes.</p>
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/useful-tips-on-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/useful-tips-on-carbondata.html b/src/main/webapp/useful-tips-on-carbondata.html
index 912cb48..eba1b0a 100644
--- a/src/main/webapp/useful-tips-on-carbondata.html
+++ b/src/main/webapp/useful-tips-on-carbondata.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,68 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__home nav__item" href="/">home</a>
+                            <a class="b-nav__intro nav__item" href="/intro">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="/quickstart">quickstart</a>
+                            <a class="b-nav__uses nav__item" href="/uses">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="/documentation">documentation</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#gettingStarted">getting started</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#api">APIs</a>
+                                <a class="b-nav__streams nav__item nav__sub__item" href="/documentation/streams">kafka streams</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#connect">kafka connect</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#configuration">configuration</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#design">design</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#implementation">implementation</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#operations">operations</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#security">security</a>
+                            </div>
+
+                            <a class="b-nav__performance nav__item" href="/performance">performance</a>
+                            <a class="b-nav__poweredby nav__item" href="/powered-by">powered by</a>
+                            <a class="b-nav__project nav__item" href="/project">project info</a>
+                            <a class="b-nav__ecosystem nav__item" href="https://cwiki.apache.org/confluence/display/KAFKA/Ecosystem" target="_blank">ecosystem</a>
+                            <a class="b-nav__clients nav__item" href="https://cwiki.apache.org/confluence/display/KAFKA/Clients" target="_blank">clients</a>
+                            <a class="b-nav__events nav__item" href="/events">events</a>
+                            <a class="b-nav__contact nav__item" href="/contact">contact us</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__apache nav__item nav__sub__anchor b-nav__sub__anchor" href="#">apache</a>
+                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/" target="_blank">foundation</a>
+                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/licenses/" target="_blank">license</a>
+                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/foundation/sponsorship.html" target="_blank">sponsorship</a>
+                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/foundation/thanks.html" target="_blank">thanks</a>
+                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/security/" target="_blank">security</a>
+                            </div>
+
+                            <a class="btn" href="/downloads">download</a>
+                            <div class="social-links">
+                                <a class="twitter" href="https://twitter.com/apachekafka" target="_blank">@apachekafka</a>
+                            </div>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__home navindicator__item"></div>
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__performance navindicator__item"></div>
+                        <div class="b-nav__poweredby navindicator__item"></div>
+                        <div class="b-nav__project navindicator__item"></div>
+                        <div class="b-nav__ecosystem navindicator__item"></div>
+                        <div class="b-nav__clients navindicator__item"></div>
+                        <div class="b-nav__events navindicator__item"></div>
+                        <div class="b-nav__contact navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -477,4 +539,4 @@ scenarios. After the completion of POC, some of the configurations impacting the
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/videogallery.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/videogallery.html b/src/main/webapp/videogallery.html
index c282364..e494174 100644
--- a/src/main/webapp/videogallery.html
+++ b/src/main/webapp/videogallery.html
@@ -246,4 +246,4 @@
 
 
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/bloomfilter-datamap-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/bloomfilter-datamap-guide.md b/src/site/markdown/bloomfilter-datamap-guide.md
index 92810f8..dd590e1 100644
--- a/src/site/markdown/bloomfilter-datamap-guide.md
+++ b/src/site/markdown/bloomfilter-datamap-guide.md
@@ -144,4 +144,16 @@ You can refer to the corresponding section in `CarbonData Lucene DataMap`.
 + In some scenarios, the BloomFilter datamap may not enhance the query performance significantly
  but if it can reduce the number of spark task,
  there is still a chance that BloomFilter datamap can enhance the performance for concurrent query.
-+ Note that BloomFilter datamap will decrease the data loading performance and may cause slightly storage expansion (for datamap index file).
\ No newline at end of file
++ Note that BloomFilter datamap will decrease the data loading performance and may cause slightly storage expansion (for datamap index file).
+
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>


[25/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/ddl-of-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/ddl-of-carbondata.html b/src/main/webapp/ddl-of-carbondata.html
index 3e15a83..2582f4d 100644
--- a/src/main/webapp/ddl-of-carbondata.html
+++ b/src/main/webapp/ddl-of-carbondata.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -296,11 +296,101 @@ STORED AS carbondata
 <p><strong>NOTE:</strong> CarbonData also supports "STORED AS carbondata" and "USING carbondata". Find example code at <a href="https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala" target=_blank>CarbonSessionExample</a> in the CarbonData repo.</p>
 <h3>
 <a id="usage-guidelines" class="anchor" href="#usage-guidelines" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
-<p><strong>Supported properties:</strong> <a href="#dictionary-encoding-configuration">DICTIONARY_INCLUDE</a>,<a href="#inverted-index-configuration">NO_INVERTED_INDEX</a>,<a href="#sort-columns-configuration">SORT_COLUMNS</a>,<a href="#sort-scope-configuration">SORT_SCOPE</a>,<a href="#table-block-size-configuration">TABLE_BLOCKSIZE</a>,<a href="#table-compaction-configuration">MAJOR_COMPACTION_SIZE</a>,</p>
-<p><a href="#table-compaction-configuration">AUTO_LOAD_MERGE</a>,<a href="#table-compaction-configuration">COMPACTION_LEVEL_THRESHOLD</a>,<a href="#table-compaction-configuration">COMPACTION_PRESERVE_SEGMENTS</a>,<a href="#table-compaction-configuration">ALLOWED_COMPACTION_DAYS</a>,</p>
-<p><a href="#streaming">streaming</a>,<a href="#local-dictionary-configuration">LOCAL_DICTIONARY_ENABLE</a>,<a href="#local-dictionary-configuration">LOCAL_DICTIONARY_THRESHOLD</a>,<a href="#local-dictionary-configuration">LOCAL_DICTIONARY_INCLUDE</a>,</p>
-<p><a href="#local-dictionary-configuration">LOCAL_DICTIONARY_EXCLUDE</a>,<a href="#caching-minmax-value-for-required-columns">COLUMN_META_CACHE</a>,<a href="#caching-at-block-or-blocklet-level">CACHE_LEVEL</a>,<a href="#support-flat-folder-same-as-hiveparquet">flat_folder</a>,<a href="#string-longer-than-32000-characters">LONG_STRING_COLUMNS</a>,<a href="#bucketing">BUCKETNUMBER</a>,</p>
-<p><a href="#bucketing">BUCKETCOLUMNS</a></p>
+<p><strong>Supported properties:</strong></p>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td><a href="#dictionary-encoding-configuration">DICTIONARY_INCLUDE</a></td>
+<td>Columns for which dictionary needs to be generated</td>
+</tr>
+<tr>
+<td><a href="#inverted-index-configuration">NO_INVERTED_INDEX</a></td>
+<td>Columns to exclude from inverted index generation</td>
+</tr>
+<tr>
+<td><a href="#sort-columns-configuration">SORT_COLUMNS</a></td>
+<td>Columns to include in sort and its order of sort</td>
+</tr>
+<tr>
+<td><a href="#sort-scope-configuration">SORT_SCOPE</a></td>
+<td>Sort scope of the load.Options include no sort, local sort ,batch sort and global sort</td>
+</tr>
+<tr>
+<td><a href="#table-block-size-configuration">TABLE_BLOCKSIZE</a></td>
+<td>Size of blocks to write onto hdfs</td>
+</tr>
+<tr>
+<td><a href="#table-compaction-configuration">MAJOR_COMPACTION_SIZE</a></td>
+<td>Size upto which the segments can be combined into one</td>
+</tr>
+<tr>
+<td><a href="#table-compaction-configuration">AUTO_LOAD_MERGE</a></td>
+<td>Whether to auto compact the segments</td>
+</tr>
+<tr>
+<td><a href="#table-compaction-configuration">COMPACTION_LEVEL_THRESHOLD</a></td>
+<td>Number of segments to compact into one segment</td>
+</tr>
+<tr>
+<td><a href="#table-compaction-configuration">COMPACTION_PRESERVE_SEGMENTS</a></td>
+<td>Number of latest segments that needs to be excluded from compaction</td>
+</tr>
+<tr>
+<td><a href="#table-compaction-configuration">ALLOWED_COMPACTION_DAYS</a></td>
+<td>Segments generated within the configured time limit in days will be compacted, skipping others</td>
+</tr>
+<tr>
+<td><a href="#streaming">streaming</a></td>
+<td>Whether the table is a streaming table</td>
+</tr>
+<tr>
+<td><a href="#local-dictionary-configuration">LOCAL_DICTIONARY_ENABLE</a></td>
+<td>Enable local dictionary generation</td>
+</tr>
+<tr>
+<td><a href="#local-dictionary-configuration">LOCAL_DICTIONARY_THRESHOLD</a></td>
+<td>Cardinality upto which the local dictionary can be generated</td>
+</tr>
+<tr>
+<td><a href="#local-dictionary-configuration">LOCAL_DICTIONARY_INCLUDE</a></td>
+<td>Columns for which local dictionary needs to be generated.Useful when local dictionary need not be generated for all string/varchar/char columns</td>
+</tr>
+<tr>
+<td><a href="#local-dictionary-configuration">LOCAL_DICTIONARY_EXCLUDE</a></td>
+<td>Columns for which local dictionary generation should be skipped.Useful when local dictionary need not be generated for few string/varchar/char columns</td>
+</tr>
+<tr>
+<td><a href="#caching-minmax-value-for-required-columns">COLUMN_META_CACHE</a></td>
+<td>Columns whose metadata can be cached in Driver for efficient pruning and improved query performance</td>
+</tr>
+<tr>
+<td><a href="#caching-at-block-or-blocklet-level">CACHE_LEVEL</a></td>
+<td>Column metadata caching level.Whether to cache column metadata of block or blocklet</td>
+</tr>
+<tr>
+<td><a href="#support-flat-folder-same-as-hiveparquet">flat_folder</a></td>
+<td>Whether to write all the carbondata files in a single folder.Not writing segments folder during incremental load</td>
+</tr>
+<tr>
+<td><a href="#string-longer-than-32000-characters">LONG_STRING_COLUMNS</a></td>
+<td>Columns which are greater than 32K characters</td>
+</tr>
+<tr>
+<td><a href="#bucketing">BUCKETNUMBER</a></td>
+<td>Number of buckets to be created</td>
+</tr>
+<tr>
+<td><a href="#bucketing">BUCKETCOLUMNS</a></td>
+<td>Columns which are to be placed in buckets</td>
+</tr>
+</tbody>
+</table>
 <p>Following are the guidelines for TBLPROPERTIES, CarbonData's additional table options can be set via carbon.properties.</p>
 <ul>
 <li>
@@ -356,15 +446,15 @@ And if you care about loading resources isolation strictly, because the system u
 <pre><code>### Example:
 </code></pre>
 <pre><code> 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'
+   productNumber INT,
+   productName STRING,
+   storeCity STRING,
+   storeProvince STRING,
+   productCategory STRING,
+   productBatch STRING,
+   saleQuantity INT,
+   revenue INT)
+ STORED AS carbondata
  TBLPROPERTIES ('SORT_COLUMNS'='productName,storeCity',
                 'SORT_SCOPE'='NO_SORT')
 </code></pre>
@@ -458,7 +548,7 @@ Following are 5 configurations:</p>
 <tr>
 <td>LOCAL_DICTIONARY_ENABLE</td>
 <td>false</td>
-<td>Whether to enable local dictionary generation. <strong>NOTE:</strong> If this property is defined, it will override the value configured at system level by 'carbon.local.dictionary.enable'</td>
+<td>Whether to enable local dictionary generation. <strong>NOTE:</strong> If this property is defined, it will override the value configured at system level by '<em><strong>carbon.local.dictionary.enable</strong></em>'.Local dictionary will be generated for all string/varchar/char columns unless LOCAL_DICTIONARY_INCLUDE, LOCAL_DICTIONARY_EXCLUDE is configured.</td>
 </tr>
 <tr>
 <td>LOCAL_DICTIONARY_THRESHOLD</td>
@@ -468,12 +558,12 @@ Following are 5 configurations:</p>
 <tr>
 <td>LOCAL_DICTIONARY_INCLUDE</td>
 <td>string/varchar/char columns</td>
-<td>Columns for which Local Dictionary has to be generated.<strong>NOTE:</strong> Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.</td>
+<td>Columns for which Local Dictionary has to be generated.<strong>NOTE:</strong> Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.This property needs to be configured only when local dictionary needs to be generated for few columns, skipping others.This property takes effect only when <strong>LOCAL_DICTIONARY_ENABLE</strong> is true or <strong>carbon.local.dictionary.enable</strong> is true</td>
 </tr>
 <tr>
 <td>LOCAL_DICTIONARY_EXCLUDE</td>
 <td>none</td>
-<td>Columns for which Local Dictionary need not be generated.</td>
+<td>Columns for which Local Dictionary need not be generated.This property needs to be configured only when local dictionary needs to be skipped for few columns, generating for others.This property takes effect only when <strong>LOCAL_DICTIONARY_ENABLE</strong> is true or <strong>carbon.local.dictionary.enable</strong> is true</td>
 </tr>
 </tbody>
 </table>
@@ -502,7 +592,7 @@ Following are 5 configurations:</p>
           
             column3 LONG )
           
-  STORED BY 'carbondata'
+  STORED AS carbondata
   TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='true','LOCAL_DICTIONARY_THRESHOLD'='1000',
   'LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
 </code></pre>
@@ -613,7 +703,7 @@ You can refer to SDKwriterTestCase for example.</p>
 <a id="create-table-as-select" class="anchor" href="#create-table-as-select" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE AS SELECT</h2>
 <p>This function allows user 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.</p>
 <pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name 
-STORED BY 'carbondata' 
+STORED AS carbondata 
 [TBLPROPERTIES (key1=val1, key2=val2, ...)] 
 AS select_statement;
 </code></pre>
@@ -629,7 +719,7 @@ carbon.sql("INSERT INTO source_table SELECT 1,'bob','shenzhen',27")
 carbon.sql("INSERT INTO source_table SELECT 2,'david','shenzhen',31")
 
 carbon.sql("CREATE TABLE target_table
-            STORED BY 'carbondata'
+            STORED AS carbondata
             AS SELECT city,avg(age) FROM source_table GROUP BY city")
             
 carbon.sql("SELECT * FROM target_table").show
@@ -645,21 +735,21 @@ carbon.sql("SELECT * FROM target_table").show
 <a id="create-external-table" class="anchor" href="#create-external-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE EXTERNAL TABLE</h2>
 <p>This function allows user to create external table by specifying location.</p>
 <pre><code>CREATE EXTERNAL TABLE [IF NOT EXISTS] [db_name.]table_name 
-STORED BY 'carbondata' LOCATION ?$FilesPath?
+STORED AS carbondata LOCATION ?$FilesPath?
 </code></pre>
 <h3>
 <a id="create-external-table-on-managed-table-data-location" class="anchor" href="#create-external-table-on-managed-table-data-location" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create external table on managed table data location.</h3>
 <p>Managed table data location provided will have both FACT and Metadata folder.
 This data can be generated by creating a normal carbon table and use this path as $FilesPath in the above syntax.</p>
 <p><strong>Example:</strong></p>
-<pre><code>sql("CREATE TABLE origin(key INT, value STRING) STORED BY 'carbondata'")
+<pre><code>sql("CREATE TABLE origin(key INT, value STRING) STORED AS carbondata")
 sql("INSERT INTO origin select 100,'spark'")
 sql("INSERT INTO origin select 200,'hive'")
 // creates a table in $storeLocation/origin
 
 sql(s"""
 |CREATE EXTERNAL TABLE source
-|STORED BY 'carbondata'
+|STORED AS carbondata
 |LOCATION '$storeLocation/origin'
 """.stripMargin)
 checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin"))
@@ -670,7 +760,7 @@ checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin
 Our SDK module currently support writing data in this format.</p>
 <p><strong>Example:</strong></p>
 <pre><code>sql(
-s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbondata' LOCATION
+s"""CREATE EXTERNAL TABLE sdkOutputTable STORED AS carbondata LOCATION
 |'$writerPath' """.stripMargin)
 </code></pre>
 <p>Here writer path will have carbondata and index files.
@@ -839,14 +929,14 @@ This can be achieved by using the alter table set command.</p>
 You can see the column comment of an existing table using describe formatted command.</p>
 <pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type [COMMENT col_comment], ...)]
   [COMMENT table_comment]
-STORED BY 'carbondata'
+STORED AS carbondata
 [TBLPROPERTIES (property_name=property_value, ...)]
 </code></pre>
 <p>Example:</p>
 <pre><code>CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
                               productNumber Int COMMENT 'unique serial number for product')
 COMMENT ?This is table comment?
- STORED BY 'carbondata'
+ STORED AS carbondata
  TBLPROPERTIES ('DICTIONARY_INCLUDE'='productNumber')
 </code></pre>
 <p>You can also SET and UNSET table comment using ALTER command.</p>
@@ -880,7 +970,7 @@ COMMENT ?This is table comment?
                               saleQuantity INT,
                               revenue INT)
 PARTITIONED BY (productCategory STRING, productBatch STRING)
-STORED BY 'carbondata'
+STORED AS carbondata
 </code></pre>
 <p>NOTE: Hive partition is not supported on complex datatype columns.</p>
 <h4>
@@ -918,7 +1008,7 @@ WHERE au.country = 'US';
 <pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
                   [(col_name data_type , ...)]
 PARTITIONED BY (partition_col_name data_type)
-STORED BY 'carbondata'
+STORED AS carbondata
 [TBLPROPERTIES ('PARTITION_TYPE'='HASH',
                 'NUM_PARTITIONS'='N' ...)]
 </code></pre>
@@ -931,7 +1021,7 @@ STORED BY 'carbondata'
     col_D DECIMAL(10,2),
     col_F TIMESTAMP
 ) PARTITIONED BY (col_E LONG)
-STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='9')
+STORED AS carbondata TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='9')
 </code></pre>
 <h3>
 <a id="create-range-partition-table" class="anchor" href="#create-range-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Range Partition Table</h3>
@@ -939,7 +1029,7 @@ STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='9
 <pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
                   [(col_name data_type , ...)]
 PARTITIONED BY (partition_col_name data_type)
-STORED BY 'carbondata'
+STORED AS carbondata
 [TBLPROPERTIES ('PARTITION_TYPE'='RANGE',
                 'RANGE_INFO'='2014-01-01, 2015-01-01, 2016-01-01, ...')]
 </code></pre>
@@ -966,7 +1056,7 @@ STORED BY 'carbondata'
 <pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
                   [(col_name data_type , ...)]
 PARTITIONED BY (partition_col_name data_type)
-STORED BY 'carbondata'
+STORED AS carbondata
 [TBLPROPERTIES ('PARTITION_TYPE'='LIST',
                 'LIST_INFO'='A, B, C, ...')]
 </code></pre>
@@ -979,7 +1069,7 @@ STORED BY 'carbondata'
     col_E LONG,
     col_F TIMESTAMP
  ) PARTITIONED BY (col_A STRING)
- STORED BY 'carbondata'
+ STORED AS carbondata
  TBLPROPERTIES('PARTITION_TYPE'='LIST',
  'LIST_INFO'='aaaa, bbbb, (cccc, dddd), eeee')
 </code></pre>
@@ -1027,7 +1117,7 @@ columns to be used for bucketing and the number of buckets. For the selection of
 of columns is used.</p>
 <pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
                   [(col_name data_type, ...)]
-STORED BY 'carbondata'
+STORED AS carbondata
 TBLPROPERTIES('BUCKETNUMBER'='noOfBuckets',
 'BUCKETCOLUMNS'='columnname')
 </code></pre>
@@ -1038,15 +1128,15 @@ TBLPROPERTIES('BUCKETNUMBER'='noOfBuckets',
 </ul>
 <p>Example:</p>
 <pre><code>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'
+  productNumber INT,
+  saleQuantity INT,
+  productName STRING,
+  storeCity STRING,
+  storeProvince STRING,
+  productCategory STRING,
+  productBatch STRING,
+  revenue INT)
+STORED AS carbondata
 TBLPROPERTIES ('BUCKETNUMBER'='4', 'BUCKETCOLUMNS'='productName')
 </code></pre>
 <script>
@@ -1059,8 +1149,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -1075,4 +1164,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/dml-of-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/dml-of-carbondata.html b/src/main/webapp/dml-of-carbondata.html
index 2048c9e..ac41f7c 100644
--- a/src/main/webapp/dml-of-carbondata.html
+++ b/src/main/webapp/dml-of-carbondata.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -231,7 +231,101 @@
 INTO TABLE [db_name.]table_name 
 OPTIONS(property_name=property_value, ...)
 </code></pre>
-<p><strong>Supported Properties:</strong> <a href="#delimiter">DELIMITER</a>, <a href="#quotechar">QUOTECHAR</a>, <a href="#commentchar">COMMENTCHAR</a>, <a href="#header">HEADER</a>, <a href="#fileheader">FILEHEADER</a>, <a href="#multiline">MULTILINE</a>, <a href="#escapechar">ESCAPECHAR</a>, <a href="#skip_empty_line">SKIP_EMPTY_LINE</a>, <a href="#complex_delimiter_level_1">COMPLEX_DELIMITER_LEVEL_1</a>, <a href="#complex_delimiter_level_2">COMPLEX_DELIMITER_LEVEL_2</a>, <a href="#all_dictionary_path">ALL_DICTIONARY_PATH</a>, <a href="#columndict">COLUMNDICT</a>, <a href="#dateformat">DATEFORMAT</a>,<a href="#timestampformat"> TIMESTAMPFORMAT</a>, <a href="#sort-column-bounds">SORT_COLUMN_BOUNDS</a>, <a href="#single_pass">SINGLE_PASS</a>, <a href="#bad-records-handling">BAD_RECORDS_LOGGER_ENABLE</a>, <a href="#bad-records-handling">BAD_RECORD_PATH</a>, <a href="#bad-records-handling">BAD_RECORDS_ACTION</a>, <a href="#bad-records-handling">IS_EMPTY_DATA_BAD_RECORD</a>, <a href="
 #global_sort_partitions">GLOBAL_SORT_PARTITIONS</a></p>
+<p><strong>Supported Properties:</strong></p>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td><a href="#delimiter">DELIMITER</a></td>
+<td>Character used to separate the data in the input csv file</td>
+</tr>
+<tr>
+<td><a href="#quotechar">QUOTECHAR</a></td>
+<td>Character used to quote the data in the input csv file</td>
+</tr>
+<tr>
+<td><a href="#commentchar">COMMENTCHAR</a></td>
+<td>Character used to comment the rows in the input csv file.Those rows will be skipped from processing</td>
+</tr>
+<tr>
+<td><a href="#header">HEADER</a></td>
+<td>Whether the input csv files have header row</td>
+</tr>
+<tr>
+<td><a href="#fileheader">FILEHEADER</a></td>
+<td>If header is not present in the input csv, what is the column names to be used for data read from input csv</td>
+</tr>
+<tr>
+<td><a href="#multiline">MULTILINE</a></td>
+<td>Whether a row data can span across multiple lines.</td>
+</tr>
+<tr>
+<td><a href="#escapechar">ESCAPECHAR</a></td>
+<td>Escape character used to excape the data in input csv file.For eg.,\ is a standard escape character</td>
+</tr>
+<tr>
+<td><a href="#skip_empty_line">SKIP_EMPTY_LINE</a></td>
+<td>Whether empty lines in input csv file should be skipped or loaded as null row</td>
+</tr>
+<tr>
+<td><a href="#complex_delimiter_level_1">COMPLEX_DELIMITER_LEVEL_1</a></td>
+<td>Starting delimiter for complex type data in input csv file</td>
+</tr>
+<tr>
+<td><a href="#complex_delimiter_level_2">COMPLEX_DELIMITER_LEVEL_2</a></td>
+<td>Ending delimiter for complex type data in input csv file</td>
+</tr>
+<tr>
+<td><a href="#all_dictionary_path">ALL_DICTIONARY_PATH</a></td>
+<td>Path to read the dictionary data from all columns</td>
+</tr>
+<tr>
+<td><a href="#columndict">COLUMNDICT</a></td>
+<td>Path to read the dictionary data from for particular column</td>
+</tr>
+<tr>
+<td><a href="#dateformat">DATEFORMAT</a></td>
+<td>Format of date in the input csv file</td>
+</tr>
+<tr>
+<td><a href="#timestampformat">TIMESTAMPFORMAT</a></td>
+<td>Format of timestamp in the input csv file</td>
+</tr>
+<tr>
+<td><a href="#sort-column-bounds">SORT_COLUMN_BOUNDS</a></td>
+<td>How to parititon the sort columns to make the evenly distributed</td>
+</tr>
+<tr>
+<td><a href="#single_pass">SINGLE_PASS</a></td>
+<td>When to enable single pass data loading</td>
+</tr>
+<tr>
+<td><a href="#bad-records-handling">BAD_RECORDS_LOGGER_ENABLE</a></td>
+<td>Whether to enable bad records logging</td>
+</tr>
+<tr>
+<td><a href="#bad-records-handling">BAD_RECORD_PATH</a></td>
+<td>Bad records logging path.Useful when bad record logging is enabled</td>
+</tr>
+<tr>
+<td><a href="#bad-records-handling">BAD_RECORDS_ACTION</a></td>
+<td>Behavior of data loading when bad record is found</td>
+</tr>
+<tr>
+<td><a href="#bad-records-handling">IS_EMPTY_DATA_BAD_RECORD</a></td>
+<td>Whether empty data of a column to be considered as bad record or not</td>
+</tr>
+<tr>
+<td><a href="#global_sort_partitions">GLOBAL_SORT_PARTITIONS</a></td>
+<td>Number of partition to use for shuffling of data during sorting</td>
+</tr>
+</tbody>
+</table>
 <p>You can use the following options to load data:</p>
 <ul>
 <li>
@@ -569,8 +663,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -585,4 +678,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/documentation.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/documentation.html b/src/main/webapp/documentation.html
index 634da94..982becf 100644
--- a/src/main/webapp/documentation.html
+++ b/src/main/webapp/documentation.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -215,13 +215,13 @@
 <p>Apache CarbonData is a new big data file format for faster interactive query using advanced columnar storage, index, compression and encoding techniques to improve computing efficiency, which helps in speeding up queries by an order of magnitude faster over PetaBytes of data.</p>
 <h2>
 <a id="getting-started" class="anchor" href="#getting-started" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Getting Started</h2>
-<p><strong>File Format Concepts:</strong> Start with the basics of understanding the <a href="./file-structure-of-carbondata.html#carbondata-file-structure">CarbonData file format</a> and its storage structure.This will help to understand other parts of the documentation, incuding deployment, programming and usage guides.</p>
+<p><strong>File Format Concepts:</strong> Start with the basics of understanding the <a href="./file-structure-of-carbondata.html#carbondata-file-format">CarbonData file format</a> and its <a href="./file-structure-of-carbondata.html">storage structure</a>.This will help to understand other parts of the documentation, including deployment, programming and usage guides.</p>
 <p><strong>Quick Start:</strong> <a href="./quick-start-guide.html#installing-and-configuring-carbondata-to-run-locally-with-spark-shell">Run an example program</a> on your local machine or <a href="https://github.com/apache/carbondata/tree/master/examples/spark2/src/main/scala/org/apache/carbondata/examples" target=_blank>study some examples</a>.</p>
 <p><strong>CarbonData SQL Language Reference:</strong> CarbonData extends the Spark SQL language and adds several <a href="./ddl-of-carbondata.html">DDL</a> and <a href="./dml-of-carbondata.html">DML</a> statements to support operations on it.Refer to the <a href="./language-manual.html">Reference Manual</a> to understand the supported features and functions.</p>
 <p><strong>Programming Guides:</strong> You can read our guides about <a href="./sdk-guide.html">APIs supported</a> to learn how to integrate CarbonData with your applications.</p>
 <h2>
-<a id="deployment" class="anchor" href="#deployment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deployment</h2>
-<p>CarbonData can be integrated with popular Execution engines like <a href="./quick-start-guide.html#spark">Spark</a> and <a href="./quick-start-guide.html#presto">Presto</a>.Refer to the <a href="./quick-start-guide.html##deployment-modes">Installation and Configuration</a> section to understand all modes of Integrating CarbonData.</p>
+<a id="integration" class="anchor" href="#integration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Integration</h2>
+<p>CarbonData can be integrated with popular Execution engines like <a href="./quick-start-guide.html#spark">Spark</a> and <a href="./quick-start-guide.html#presto">Presto</a>.Refer to the <a href="./quick-start-guide.html#integration">Installation and Configuration</a> section to understand all modes of Integrating CarbonData.</p>
 <h2>
 <a id="contributing-to-carbondata" class="anchor" href="#contributing-to-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contributing to CarbonData</h2>
 <p>The Apache CarbonData community welcomes all kinds of contributions from anyone with a passion for
@@ -234,10 +234,6 @@ faster data format.Contributing to CarbonData doesn?t just mean writing code. He
 <p><strong>Blogs:</strong> Blogs by external users can be found <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=67635497" rel="nofollow">here</a>.</p>
 <p><strong>Performance reports:</strong> TPC-H performance reports can be found <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Performance+-+TPCH+Report+of+CarbonData+%281.2+version%29+and+Parquet+on+Spark+Execution+Engine" rel="nofollow">here</a>.</p>
 <p><strong>Trainings:</strong> Training records on design and code flows can be found <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/CarbonData+Training+Materials" rel="nofollow">here</a>.</p>
-<script>
-// Show selected style on nav item
-$(function() { $('.b-nav__intro').addClass('selected'); });
-</script>
 </div>
 </div>
 </div>
@@ -253,4 +249,4 @@ $(function() { $('.b-nav__intro').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/errorpage.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/errorpage.html b/src/main/webapp/errorpage.html
index 35cc01a..090dce5 100644
--- a/src/main/webapp/errorpage.html
+++ b/src/main/webapp/errorpage.html
@@ -89,4 +89,4 @@
 
 
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/faq.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/faq.html b/src/main/webapp/faq.html
index 52112b7..c37284f 100644
--- a/src/main/webapp/faq.html
+++ b/src/main/webapp/faq.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -222,19 +222,19 @@
 <li><a href="#how-to-resolve-abstract-method-error">How to resolve Abstract Method Error?</a></li>
 <li><a href="#how-carbon-will-behave-when-execute-insert-operation-in-abnormal-scenarios">How Carbon will behave when execute insert operation in abnormal scenarios?</a></li>
 <li><a href="#why-aggregate-query-is-not-fetching-data-from-aggregate-table">Why aggregate query is not fetching data from aggregate table?</a></li>
-<li><a href="#Why-all-executors-are-showing-success-in-Spark-UI-even-after-Dataload-command-failed-at-driver-side">Why all executors are showing success in Spark UI even after Dataload command failed at Driver side?</a></li>
-<li><a href="#Why-different-time-zone-result-for-select-query-output-when-query-SDK-writer-output">Why different time zone result for select query output when query SDK writer output?</a></li>
+<li><a href="#why-all-executors-are-showing-success-in-spark-ui-even-after-dataload-command-failed-at-driver-side">Why all executors are showing success in Spark UI even after Dataload command failed at Driver side?</a></li>
+<li><a href="#why-different-time-zone-result-for-select-query-output-when-query-sdk-writer-output">Why different time zone result for select query output when query SDK writer output?</a></li>
 </ul>
 <h1>
 <a id="troubleshooting" class="anchor" href="#troubleshooting" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>TroubleShooting</h1>
 <ul>
-<li><a href="#Getting-tablestatus.lock-issues-When-loading-data">Getting tablestatus.lock issues When loading data</a></li>
+<li><a href="#getting-tablestatuslock-issues-when-loading-data">Getting tablestatus.lock issues When loading data</a></li>
 <li><a href="#failed-to-load-thrift-libraries">Failed to load thrift libraries</a></li>
 <li><a href="#failed-to-launch-the-spark-shell">Failed to launch the Spark Shell</a></li>
 <li><a href="#failed-to-execute-load-query-on-cluster">Failed to execute load query on cluster</a></li>
 <li><a href="#failed-to-execute-insert-query-on-cluster">Failed to execute insert query on cluster</a></li>
 <li><a href="#failed-to-connect-to-hiveuser-with-thrift">Failed to connect to hiveuser with thrift</a></li>
-<li><a href="#failed-to-read-the-metastore-db-during-table">Failed to read the metastore db during table</a></li>
+<li><a href="#failed-to-read-the-metastore-db-during-table-creation">Failed to read the metastore db during table creation</a></li>
 <li><a href="#failed-to-load-data-on-the-cluster">Failed to load data on the cluster</a></li>
 <li><a href="#failed-to-insert-data-on-the-cluster">Failed to insert data on the cluster</a></li>
 <li><a href="#failed-to-execute-concurrent-operations-on-table-by-multiple-workers">Failed to execute Concurrent Operations(Load,Insert,Update) on table by multiple workers</a></li>
@@ -292,7 +292,7 @@ The property carbon.lock.type configuration specifies the type of lock to be acq
 <p>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 <code>spark version</code> while using Maven to build project.</p>
 <h2>
 <a id="how-carbon-will-behave-when-execute-insert-operation-in-abnormal-scenarios" class="anchor" href="#how-carbon-will-behave-when-execute-insert-operation-in-abnormal-scenarios" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>How Carbon will behave when execute insert operation in abnormal scenarios?</h2>
-<p>Carbon support insert operation, you can refer to the syntax mentioned in <a href="dml-operation-on-carbondata.html">DML Operations on CarbonData</a>.
+<p>Carbon support insert operation, you can refer to the syntax mentioned in <a href="./dml-of-carbondata.html">DML Operations on CarbonData</a>.
 First, create a source table in spark-sql and load data into this created table.</p>
 <pre><code>CREATE TABLE source_table(
 id String,
@@ -312,7 +312,7 @@ id  name    city
 id String,
 city String,
 name String)
-STORED BY 'carbondata';
+STORED AS carbondata;
 </code></pre>
 <pre><code>INSERT INTO TABLE carbon_table SELECT * FROM source_table;
 </code></pre>
@@ -341,7 +341,7 @@ id  city    name
 When SubQuery predicate is present in the query.</li>
 </ul>
 <p>Example:</p>
-<pre><code>create table gdp21(cntry smallint, gdp double, y_year date) stored by 'carbondata';
+<pre><code>create table gdp21(cntry smallint, gdp double, y_year date) stored as carbondata;
 create datamap ag1 on table gdp21 using 'preaggregate' as select cntry, sum(gdp) from gdp21 group by cntry;
 select ctry from pop1 where ctry in (select cntry from gdp21 group by cntry);
 </code></pre>
@@ -351,7 +351,7 @@ select ctry from pop1 where ctry in (select cntry from gdp21 group by cntry);
 When aggregate function along with 'in' filter.</li>
 </ul>
 <p>Example:</p>
-<pre><code>create table gdp21(cntry smallint, gdp double, y_year date) stored by 'carbondata';
+<pre><code>create table gdp21(cntry smallint, gdp double, y_year date) stored as carbondata;
 create datamap ag1 on table gdp21 using 'preaggregate' as select cntry, sum(gdp) from gdp21 group by cntry;
 select cntry, sum(gdp) from gdp21 where cntry in (select ctry from pop1) group by cntry;
 </code></pre>
@@ -361,7 +361,7 @@ select cntry, sum(gdp) from gdp21 where cntry in (select ctry from pop1) group b
 When aggregate function having 'join' with equal filter.</li>
 </ul>
 <p>Example:</p>
-<pre><code>create table gdp21(cntry smallint, gdp double, y_year date) stored by 'carbondata';
+<pre><code>create table gdp21(cntry smallint, gdp double, y_year date) stored as carbondata;
 create datamap ag1 on table gdp21 using 'preaggregate' as select cntry, sum(gdp) from gdp21 group by cntry;
 select cntry,sum(gdp) from gdp21,pop1 where cntry=ctry group by cntry;
 </code></pre>
@@ -566,8 +566,7 @@ For example, you can use scp to copy this file to all the nodes.</p>
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__faq').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -582,4 +581,4 @@ $(function() { $('.b-nav__faq').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/file-structure-of-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/file-structure-of-carbondata.html b/src/main/webapp/file-structure-of-carbondata.html
index b96d622..c14ea6d 100644
--- a/src/main/webapp/file-structure-of-carbondata.html
+++ b/src/main/webapp/file-structure-of-carbondata.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -211,30 +211,138 @@
                                 <div class="col-sm-12  col-md-12">
                                     <div>
 <h1>
-<a id="carbondata-file-structure" class="anchor" href="#carbondata-file-structure" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData File Structure</h1>
+<a id="carbondata-table-structure" class="anchor" href="#carbondata-table-structure" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData table structure</h1>
 <p>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.</p>
 <p>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.</p>
-<h3>
-<a id="understanding-carbondata-file-structure" class="anchor" href="#understanding-carbondata-file-structure" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Understanding CarbonData File Structure</h3>
+<p>This document describes the what a CarbonData table looks like in a HDFS directory, files written and content of each file.</p>
 <ul>
-<li>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.</li>
-</ul>
-<p><a href="../docs/images/carbon_data_file_structure_new.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_file_structure_new.png?raw=true" alt="CarbonData File Structure" style="max-width:100%;"></a></p>
+<li>
+<p><a href="#file-directory-structure">File Directory Structure</a></p>
+</li>
+<li>
+<p><a href="#file-content-details">File Content details</a></p>
 <ul>
-<li>File Header : It contains CarbonData file version number, list of column schema and schema updation timestamp.</li>
-<li>File Footer : it contains Number of rows, segmentinfo ,all blocklets? info and index, you can find the detail from the below diagram.</li>
-<li>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.</li>
-<li>Column Page Group : Data of one column and it is further divided into pages, it is guaranteed to be contiguous in file.</li>
-<li>Page : It has the data of one column and the number of row is fixed to 32000 size.</li>
-</ul>
-<p><a href="../docs/images/carbon_data_format_new.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_format_new.png?raw=true" alt="CarbonData File Format" style="max-width:100%;"></a></p>
-<h3>
-<a id="each-page-contains-three-types-of-data" class="anchor" href="#each-page-contains-three-types-of-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Each page contains three types of data</h3>
+<li><a href="#schema-file-format">Schema file format</a></li>
+<li>
+<a href="#carbondata-file-format">CarbonData file format</a>
 <ul>
-<li>Data Page: Contains the encoded data of a column of columns.</li>
-<li>Row ID Page (optional): Contains the row ID mappings used when the data page is stored as an inverted index.</li>
-<li>RLE Page (optional): Contains additional metadata used when the data page is RLE coded.</li>
+<li>
+<a href="#blocklet-format">Blocklet format</a>
+<ul>
+<li><a href="#v1">V1</a></li>
+<li><a href="#v2">V2</a></li>
+<li><a href="#v3">V3</a></li>
+</ul>
+</li>
+<li><a href="#footer-format">Footer format</a></li>
 </ul>
+</li>
+<li><a href="#carbonindex-file-format">carbonindex file format</a></li>
+<li><a href="#dictionary-file-format">Dictionary file format</a></li>
+<li><a href="#tablestatus-file-format">tablestatus file format</a></li>
+</ul>
+</li>
+</ul>
+<h2>
+<a id="file-directory-structure" class="anchor" href="#file-directory-structure" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>File Directory Structure</h2>
+<p>The CarbonData files are stored in the location specified by the <em><strong>carbon.storelocation</strong></em> configuration (configured in carbon.properties; if not configured, the default is ../carbon.store).</p>
+<p>The file directory structure is as below:</p>
+<p><a href="../docs/images/2-1_1.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-1_1.png?raw=true" alt="File Directory Structure" style="max-width:100%;"></a></p>
+<ol>
+<li>ModifiedTime.htmlt records the timestamp of the metadata with the modification time attribute of the file. When the drop table and create table are used, the modification time of the file is updated.This is common to all databases and hence is kept in parallel to databases</li>
+<li>The <strong>default</strong> is the database name and contains the user tables.default is used when user doesn't specify any database name;else user configured database name will be the directory name. user_table is the table name.</li>
+<li>Metadata directory stores schema files, tablestatus and dictionary files (including .dict, .dictmeta and .sortindex). There are three types of metadata data information files.</li>
+<li>data and index files are stored under directory named <strong>Fact</strong>. The Fact directory has a Part0 partition directory, where 0 is the partition number.</li>
+<li>There is a Segment_0 directory under the Part0 directory, where 0 is the segment number.</li>
+<li>There are two types of files, carbondata and carbonindex, in the Segment_0 directory.</li>
+</ol>
+<h2>
+<a id="file-content-details" class="anchor" href="#file-content-details" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>File Content details</h2>
+<p>When the table is created, the user_table directory is generated, and a schema file is generated in the Metadata directory for recording the table structure.</p>
+<p>When loading data in batches, each batch loading generates a new segment directory. The scheduling tries to control a task processing data loading task on each node. Each task will generate multiple carbondata files and one carbonindex file.</p>
+<p>During  global dictionary generation, if the two-pass scheme is used, before the data is loaded, the corresponding dict, dictmeta and sortindex files are generated for each dictionary-encoded column, and partial dictionary files can be provided by the pre-define dictionary method to reduce the need. A dictionary-encoded column is generated by scanning the full amount of data; a dictionary file of all dictionary code columns can also be provided by the all dictionary method to avoid scanning data. If the single-pass scheme is adopted, the global dictionary code is generated in real time during data loading, and after the data is loaded, the dictionary is solidified into a dictionary file.</p>
+<p>The following sections use the Java object generated by the thrift file describing the carbondata file format to explain the contents of each file one by one (you can also directly read the format defined in the <a href="https://github.com/apache/carbondata/tree/master/format/src/main/thrift" target=_blank>thrift file</a>)</p>
+<h3>
+<a id="schema-file-format" class="anchor" href="#schema-file-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Schema file format</h3>
+<p>The contents of the schema file is as shown below</p>
+<p><a href="../docs/images/2-2_1.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-2_1.png?raw=true" alt="Schema file format" style="max-width:100%;"></a></p>
+<ol>
+<li>TableSchema class
+The TableSchema class does not store the table name, it is infered from the directory name(user_table).
+tableProperties is used to record table-related properties, such as: table_blocksize.</li>
+<li>ColumnSchema class
+Encoders are used to record the encoding used in column storage.
+columnProperties is used to record column related properties.</li>
+<li>BucketingInfo class
+When creating a bucket table, you can specify the number of buckets in the table and the column to splitbuckets.</li>
+<li>DataType class
+Describes the data types supported by CarbonData.</li>
+<li>Encoding class
+Several encodings that may be used in CarbonData files.</li>
+</ol>
+<h3>
+<a id="carbondata-file-format" class="anchor" href="#carbondata-file-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData file format</h3>
+<h4>
+<a id="file-header" class="anchor" href="#file-header" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>File Header</h4>
+<p>It contains CarbonData file version number, list of column schema and schema updation timestamp.</p>
+<p><a href="../docs/images/carbon_data_file_structure_new.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_file_structure_new.png?raw=true" alt="File Header" style="max-width:100%;"></a></p>
+<p>The carbondata file consists of multiple blocklets and footer parts. The blocklet is the dataset inside the carbondata file (the latest V3 format, the default configuration is 64MB), each blocklet contains a ColumnChunk for each column, and a ColumnChunk may contain one or more Column Pages.</p>
+<p>The carbondata file currently supports V1, V2 and V3 versions. The main difference is the change of the blocklet part, which is introduced one by one.</p>
+<h4>
+<a id="blocklet-format" class="anchor" href="#blocklet-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Blocklet format</h4>
+<h5>
+<a id="v1" class="anchor" href="#v1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>V1</h5>
+<p>Blocket consists of all column data pages, RLE pages, and rowID pages. Since the pages in the blocklet are grouped according to the page type, the three pieces of data of each column are distributed and stored in the blocklet, and the offset and length information of all the pages need to be recorded in the footer part.</p>
+<p><a href="../docs/images/2-3_1.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-3_1.png?raw=true" alt="V1" style="max-width:100%;"></a></p>
+<h5>
+<a id="v2" class="anchor" href="#v2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>V2</h5>
+<p>The blocklet consists of ColumnChunk for all columns. The ColumnChunk for a column consists of a ColumnPage, which includes the data chunk header, data page, RLE page, and rowID page. Since ColumnChunk aggregates the three types of Page data of the column together, it can read the column data using fewer readers. Since the header part records the length information of all the pages, the footer part only needs to record the offset and length of the ColumnChunk, and also reduces the amount of footer data.</p>
+<p><a href="../docs/images/2-3_2.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-3_2.png?raw=true" alt="V2" style="max-width:100%;"></a></p>
+<h5>
+<a id="v3" class="anchor" href="#v3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>V3</h5>
+<p>The blocklet is also composed of ColumnChunks of all columns. What is changed is that a ColumnChunk consists of one or more Column Pages, and Column Page adds a new BlockletMinMaxIndex.</p>
+<p>Compared with V2: The blocklet data volume of V2 format defaults to 120,000 lines, and the blocklet data volume of V3 format defaults to 64MB. For the same size data file, the information of the footer part index metadata may be further reduced; meanwhile, the V3 format adds a new page. Level data filtering, and the amount of data per page is only 32,000 lines by default, which is much less than the 120,000 lines of V2 format. The accuracy of data filtering hits further, and more data can be filtered out before decompressing data.</p>
+<p><a href="../docs/images/2-3_3.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-3_3.png?raw=true" alt="V3" style="max-width:100%;"></a></p>
+<h4>
+<a id="footer-format" class="anchor" href="#footer-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Footer format</h4>
+<p>Footer records each carbondata
+All blocklet data distribution information and statistical related metadata information (minmax, startkey/endkey) inside the file.</p>
+<p><a href="../docs/images/2-3_4.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-3_4.png?raw=true" alt="Footer format" style="max-width:100%;"></a></p>
+<ol>
+<li>BlockletInfo3 is used to record the offset and length of all ColumnChunk3.</li>
+<li>SegmentInfo is used to record the number of columns and the cardinality of each column.</li>
+<li>BlockletIndex includes BlockletMinMaxIndex and BlockletBTreeIndex.</li>
+</ol>
+<p>BlockletBTreeIndex is used to record the startkey/endkey of all blocklets in the block. When querying, the startkey/endkey of the query is generated by filtering conditions combined with.htmlkey. With BlocketBtreeIndex, the range of blocklets satisfying the conditions in each block can be delineated.</p>
+<p>BlockletMinMaxIndex is used to record the min/max value of all columns in the blocklet. By using the min/max check on the filter condition, you can skip the block/blocklet that does not satisfy the condition.</p>
+<h3>
+<a id="carbonindex-file-format" class="anchor" href="#carbonindex-file-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>carbonindex file format</h3>
+<p>Extract the BlockletIndex part of the footer part to generate the carbonindex file. Load data in batches, schedule as much as possible to control a node to start a task, each task generates multiple carbondata files and a carbonindex file. The carbonindex file records the index information of all the blocklets in all the carbondata files generated by the task.</p>
+<p>As shown in the figure, the index information corresponding to a block is recorded by a BlockIndex object, including carbondata filename, footer offset and BlockletIndex. The BlockIndex data volume is less than the footer. The file is directly used to build the index on the driver side when querying, without having to skip the footer part of the data volume of multiple data files.</p>
+<p><a href="../docs/images/2-4_1.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-4_1.png?raw=true" alt="carbonindex file format" style="max-width:100%;"></a></p>
+<h3>
+<a id="dictionary-file-format" class="anchor" href="#dictionary-file-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Dictionary file format</h3>
+<p>For each dictionary encoded column, a dictionary file is used to store the dictionary metadata for that column.</p>
+<ol>
+<li>dict file records the distinct value list of a column</li>
+</ol>
+<p>For the first time dataloading, the file is generated using a distinct value list of a column. The value in the file is unordered; the subsequent append is used. In the second step of dataloading (Data Convert Step), the dictionary code column will replace the true value of the data with the dictionary key.</p>
+<p><a href="../docs/images/2-5_1.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-5_1.png?raw=true" alt="Dictionary file format" style="max-width:100%;"></a></p>
+<ol start="2">
+<li>dictmeta records the metadata description of the new distinct value of each dataloading</li>
+</ol>
+<p>The dictionary cache uses this information to incrementally flush the cache.</p>
+<p><a href="../docs/images/2-5_2.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-5_2.png?raw=true" alt="Dictionary Chunk" style="max-width:100%;"></a></p>
+<ol start="3">
+<li>sortindex records the result set of the key code of the dictionary code sorted by value.</li>
+</ol>
+<p>In dataLoading, if there is a new dictionary value, the sortindex file will be regenerated using all the dictionary codes.</p>
+<p>Filtering queries based on dictionary code columns need to convert the value filter filter to the key filter condition. Using the sortindex file, you can quickly construct an ordered value sequence to quickly find the key value corresponding to the value, thus speeding up the conversion process.</p>
+<p><a href="../docs/images/2-5_3.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-5_3.png?raw=true" alt="sortindex file format" style="max-width:100%;"></a></p>
+<h3>
+<a id="tablestatus-file-format" class="anchor" href="#tablestatus-file-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>tablestatus file format</h3>
+<p>Tablestatus records the segment-related information (in gson format) for each load and merge, including load time, load status, segment name, whether it was deleted, and the segment name incorporated. Regenerate the tablestatusfile after each load or merge.</p>
+<p><a href="../docs/images/2-6_1.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-6_1.png?raw=true" alt="tablestatus file format" style="max-width:100%;"></a></p>
 <script>
 $(function() {
   // Show selected style on nav item
@@ -245,8 +353,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -261,4 +368,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/how-to-contribute-to-apache-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/how-to-contribute-to-apache-carbondata.html b/src/main/webapp/how-to-contribute-to-apache-carbondata.html
index 946013f..122b763 100644
--- a/src/main/webapp/how-to-contribute-to-apache-carbondata.html
+++ b/src/main/webapp/how-to-contribute-to-apache-carbondata.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -352,8 +352,7 @@ $ git push &lt;GitHub_user&gt; --delete &lt;my-branch&gt;
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__contri').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -368,4 +367,4 @@ $(function() { $('.b-nav__contri').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/index.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/index.html b/src/main/webapp/index.html
index e9e6e3b..f059d16 100644
--- a/src/main/webapp/index.html
+++ b/src/main/webapp/index.html
@@ -477,7 +477,7 @@
                         </ol>
                         <p class="title-info">
                             For detailed reference on CarbonData, read the <a
-                                href="mainpage.html">User Guide</a>.
+                                href="documentation.html">User Guide</a>.
                         </p>
                     </div>
 


[26/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/data-management-on-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/data-management-on-carbondata.html b/src/main/webapp/data-management-on-carbondata.html
deleted file mode 100644
index bb5ae78..0000000
--- a/src/main/webapp/data-management-on-carbondata.html
+++ /dev/null
@@ -1,1321 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
-                                   target="_blank">Apache CarbonData 1.4.1</a></li>
-							<li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
-                                   target="_blank">Apache CarbonData 1.4.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
-                                   target="_blank">Apache CarbonData 1.3.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
-                                   target="_blank">Apache CarbonData 1.3.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search" target="_blank">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input"  placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="row">
-                                <div class="col-sm-12  col-md-12">
-                                    <div>
-<h1>
-<a id="data-management-on-carbondata" class="anchor" href="#data-management-on-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Management on CarbonData</h1>
-<p>This tutorial is going to introduce all commands and data operations on CarbonData.</p>
-<ul>
-<li><a href="#create-table">CREATE TABLE</a></li>
-<li><a href="#create-database">CREATE DATABASE</a></li>
-<li><a href="#table-management">TABLE MANAGEMENT</a></li>
-<li><a href="#load-data">LOAD DATA</a></li>
-<li><a href="#update-and-delete">UPDATE AND DELETE</a></li>
-<li><a href="#compaction">COMPACTION</a></li>
-<li><a href="#partition">PARTITION</a></li>
-<li><a href="#bucketing">BUCKETING</a></li>
-<li><a href="#segment-management">SEGMENT MANAGEMENT</a></li>
-</ul>
-<h2>
-<a id="create-table" class="anchor" href="#create-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE</h2>
-<p>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.</p>
-<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type , ...)]
-STORED AS carbondata
-[TBLPROPERTIES (property_name=property_value, ...)]
-[LOCATION 'path']
-</code></pre>
-<p><strong>NOTE:</strong> CarbonData also supports "STORED AS carbondata" and "USING carbondata". Find example code at <a href="https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala" target=_blank>CarbonSessionExample</a> in the CarbonData repo.</p>
-<h3>
-<a id="usage-guidelines" class="anchor" href="#usage-guidelines" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
-<p>Following are the guidelines for TBLPROPERTIES, CarbonData's additional table options can be set via carbon.properties.</p>
-<ul>
-<li>
-<p><strong>Dictionary Encoding Configuration</strong></p>
-<p>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.</p>
-<pre><code>TBLPROPERTIES ('DICTIONARY_INCLUDE'='column1, column2')
-</code></pre>
-<p>NOTE: Dictionary Include/Exclude for complex child columns is not supported.</p>
-</li>
-<li>
-<p><strong>Inverted Index Configuration</strong></p>
-<p>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.</p>
-<pre><code>TBLPROPERTIES ('NO_INVERTED_INDEX'='column1, column3')
-</code></pre>
-</li>
-<li>
-<p><strong>Sort Columns Configuration</strong></p>
-<p>This property is for users to specify which columns belong to the MDK(Multi-Dimensions-Key) index.</p>
-<ul>
-<li>If users don't specify "SORT_COLUMN" property, by default MDK index be built by using all dimension columns except complex data type column.</li>
-<li>If this property is specified but with empty argument, then the table will be loaded without sort.</li>
-<li>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.</li>
-</ul>
-<pre><code>TBLPROPERTIES ('SORT_COLUMNS'='column1, column3')
-OR
-TBLPROPERTIES ('SORT_COLUMNS'='')
-</code></pre>
-<p>NOTE: Sort_Columns for Complex datatype columns is not supported.</p>
-</li>
-<li>
-<p><strong>Sort Scope Configuration</strong></p>
-<p>This property is for users to specify the scope of the sort during data load, following are the types of sort scope.</p>
-<ul>
-<li>LOCAL_SORT: It is the default sort scope.</li>
-<li>NO_SORT: It will load the data in unsorted manner, it will significantly increase load performance.</li>
-<li>BATCH_SORT: It increases the load performance but decreases the query performance if identified blocks &gt; parallelism.</li>
-<li>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.</li>
-</ul>
-</li>
-</ul>
-<pre><code>### Example:
-</code></pre>
-<pre><code> 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 ('SORT_COLUMNS'='productName,storeCity',
-                'SORT_SCOPE'='NO_SORT')
-</code></pre>
-<p><strong>NOTE:</strong> CarbonData also supports "using carbondata". Find example code at <a href="https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala" target=_blank>SparkSessionExample</a> in the CarbonData repo.</p>
-<ul>
-<li>
-<p><strong>Table Block Size Configuration</strong></p>
-<p>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.</p>
-<pre><code>TBLPROPERTIES ('TABLE_BLOCKSIZE'='512')
-</code></pre>
-<p><strong>NOTE:</strong> 512 or 512M both are accepted.</p>
-</li>
-<li>
-<p><strong>Table Compaction Configuration</strong></p>
-<p>These properties are table level compaction configurations, if not specified, system level configurations in carbon.properties will be used.
-Following are 5 configurations:</p>
-<ul>
-<li>MAJOR_COMPACTION_SIZE: same meaning as carbon.major.compaction.size, size in MB.</li>
-<li>AUTO_LOAD_MERGE: same meaning as carbon.enable.auto.load.merge.</li>
-<li>COMPACTION_LEVEL_THRESHOLD: same meaning as carbon.compaction.level.threshold.</li>
-<li>COMPACTION_PRESERVE_SEGMENTS: same meaning as carbon.numberof.preserve.segments.</li>
-<li>ALLOWED_COMPACTION_DAYS: same meaning as carbon.allowed.compaction.days.</li>
-</ul>
-<pre><code>TBLPROPERTIES ('MAJOR_COMPACTION_SIZE'='2048',
-               'AUTO_LOAD_MERGE'='true',
-               'COMPACTION_LEVEL_THRESHOLD'='5,6',
-               'COMPACTION_PRESERVE_SEGMENTS'='10',
-               'ALLOWED_COMPACTION_DAYS'='5')
-</code></pre>
-</li>
-<li>
-<p><strong>Streaming</strong></p>
-<p>CarbonData supports streaming ingestion for real-time data. You can create the ?streaming? table using the following table properties.</p>
-<pre><code>TBLPROPERTIES ('streaming'='true')
-</code></pre>
-</li>
-<li>
-<p><strong>Local Dictionary Configuration</strong></p>
-</li>
-</ul>
-<p>Columns for which dictionary is not generated needs more storage space and in turn more IO. Also since more data will have to be read during query, query performance also would suffer.Generating dictionary per blocklet for such columns would help in saving storage space and assist in improving query performance as carbondata is optimized for handling dictionary encoded columns more effectively.Generating dictionary internally per blocklet is termed as local dictionary. Please refer to <a href="../file-structure-of-carbondata.html">File structure of Carbondata</a> for understanding about the file structure of carbondata and meaning of terms like blocklet.</p>
-<p>Local Dictionary helps in:</p>
-<ol>
-<li>Getting more compression.</li>
-<li>Filter queries and full scan queries will be faster as filter will be done on encoded data.</li>
-<li>Reducing the store size and memory footprint as only unique values will be stored as part of local dictionary and corresponding data will be stored as encoded data.</li>
-<li>Getting higher IO throughput.</li>
-</ol>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>
-<p>Following Data Types are Supported for Local Dictionary:</p>
-<ul>
-<li>STRING</li>
-<li>VARCHAR</li>
-<li>CHAR</li>
-</ul>
-</li>
-<li>
-<p>Following Data Types are not Supported for Local Dictionary:</p>
-<ul>
-<li>SMALLINT</li>
-<li>INTEGER</li>
-<li>BIGINT</li>
-<li>DOUBLE</li>
-<li>DECIMAL</li>
-<li>TIMESTAMP</li>
-<li>DATE</li>
-<li>BOOLEAN</li>
-</ul>
-</li>
-<li>
-<p>In case of multi-level complex dataType columns, primitive string/varchar/char columns are considered for local dictionary generation.</p>
-</li>
-</ul>
-<p>Local dictionary will have to be enabled explicitly during create table or by enabling the system property 'carbon.local.dictionary.enable'. By default, Local Dictionary will be disabled for the carbondata table.</p>
-<p>Local Dictionary can be configured using the following properties during create table command:</p>
-<table>
-<thead>
-<tr>
-<th>Properties</th>
-<th>Default value</th>
-<th>Description</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>LOCAL_DICTIONARY_ENABLE</td>
-<td>false</td>
-<td>Whether to enable local dictionary generation. <strong>NOTE:</strong> If this property is defined, it will override the value configured at system level by 'carbon.local.dictionary.enable'</td>
-</tr>
-<tr>
-<td>LOCAL_DICTIONARY_THRESHOLD</td>
-<td>10000</td>
-<td>The maximum cardinality of a column upto which carbondata can try to generate local dictionary (maximum - 100000)</td>
-</tr>
-<tr>
-<td>LOCAL_DICTIONARY_INCLUDE</td>
-<td>string/varchar/char columns</td>
-<td>Columns for which Local Dictionary has to be generated.<strong>NOTE:</strong> Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.</td>
-</tr>
-<tr>
-<td>LOCAL_DICTIONARY_EXCLUDE</td>
-<td>none</td>
-<td>Columns for which Local Dictionary need not be generated.</td>
-</tr>
-</tbody>
-</table>
-<p><strong>Fallback behavior:</strong></p>
-<ul>
-<li>When the cardinality of a column exceeds the threshold, it triggers a fallback and the generated dictionary will be reverted and data loading will be continued without dictionary encoding.</li>
-</ul>
-<p><strong>NOTE:</strong> When fallback is triggered, the data loading performance will decrease as encoded data will be discarded and the actual data is written to the temporary sort files.</p>
-<p><strong>Points to be noted:</strong></p>
-<ol>
-<li>
-<p>Reduce Block size:</p>
-<p>Number of Blocks generated is less in case of Local Dictionary as compression ratio is high. This may reduce the number of tasks launched during query, resulting in degradation of query performance if the pruned blocks are less compared to the number of parallel tasks which can be run. So it is recommended to configure smaller block size which in turn generates more number of blocks.</p>
-</li>
-<li>
-<p>All the page-level data for a blocklet needs to be maintained in memory until all the pages encoded for local dictionary is processed in order to handle fallback. Hence the memory required for local dictionary based table is more and this memory increase is proportional to number of columns.</p>
-</li>
-</ol>
-<h3>
-<a id="example" class="anchor" href="#example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
-<pre><code>CREATE TABLE carbontable(
-          
-            column1 string,
-          
-            column2 string,
-          
-            column3 LONG )
-          
-  STORED BY 'carbondata'
-  TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='true','LOCAL_DICTIONARY_THRESHOLD'='1000',
-  'LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>We recommend to use Local Dictionary when cardinality is high but is distributed across multiple loads</li>
-<li>On a large cluster, decoding data can become a bottleneck for global dictionary as there will be many remote reads. In this scenario, it is better to use Local Dictionary.</li>
-<li>When cardinality is less, but loads are repetitive, it is better to use global dictionary as local dictionary generates multiple dictionary files at blocklet level increasing redundancy.</li>
-</ul>
-<ul>
-<li>
-<p><strong>Caching Min/Max Value for Required Columns</strong>
-By default, CarbonData caches min and max values of all the columns in schema.  As the load increases, the memory required to hold the min and max values increases considerably. This feature enables you to configure min and max values only for the required columns, resulting in optimized memory usage.</p>
-<p>Following are the valid values for COLUMN_META_CACHE:</p>
-<ul>
-<li>If you want no column min/max values to be cached in the driver.</li>
-</ul>
-<pre><code>COLUMN_META_CACHE=??
-</code></pre>
-<ul>
-<li>If you want only col1 min/max values to be cached in the driver.</li>
-</ul>
-<pre><code>COLUMN_META_CACHE=?col1?
-</code></pre>
-<ul>
-<li>If you want min/max values to be cached in driver for all the specified columns.</li>
-</ul>
-<pre><code>COLUMN_META_CACHE=?col1,col2,col3,??
-</code></pre>
-<p>Columns to be cached can be specified either while creating table or after creation of the table.
-During create table operation; specify the columns to be cached in table properties.</p>
-<p>Syntax:</p>
-<pre><code>CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,?) STORED BY ?carbondata? TBLPROPERTIES (?COLUMN_META_CACHE?=?col1,col2,??)
-</code></pre>
-<p>Example:</p>
-<pre><code>CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES (?COLUMN_META_CACHE?=?name?)
-</code></pre>
-<p>After creation of table or on already created tables use the alter table command to configure the columns to be cached.</p>
-<p>Syntax:</p>
-<pre><code>ALTER TABLE [dbName].tableName SET TBLPROPERTIES (?COLUMN_META_CACHE?=?col1,col2,??)
-</code></pre>
-<p>Example:</p>
-<pre><code>ALTER TABLE employee SET TBLPROPERTIES (?COLUMN_META_CACHE?=?city?)
-</code></pre>
-</li>
-<li>
-<p><strong>Caching at Block or Blocklet Level</strong></p>
-<p>This feature allows you to maintain the cache at Block level, resulting in optimized usage of the memory. The memory consumption is high if the Blocklet level caching is maintained as a Block can have multiple Blocklet.</p>
-<p>Following are the valid values for CACHE_LEVEL:</p>
-<p><em>Configuration for caching in driver at Block level (default value).</em></p>
-<pre><code>CACHE_LEVEL= ?BLOCK?
-</code></pre>
-<p><em>Configuration for caching in driver at Blocklet level.</em></p>
-<pre><code>CACHE_LEVEL= ?BLOCKLET?
-</code></pre>
-<p>Cache level can be specified either while creating table or after creation of the table.
-During create table operation specify the cache level in table properties.</p>
-<p>Syntax:</p>
-<pre><code>CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,?) STORED BY ?carbondata? TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
-</code></pre>
-<p>Example:</p>
-<pre><code>CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
-</code></pre>
-<p>After creation of table or on already created tables use the alter table command to configure the cache level.</p>
-<p>Syntax:</p>
-<pre><code>ALTER TABLE [dbName].tableName SET TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
-</code></pre>
-<p>Example:</p>
-<pre><code>ALTER TABLE employee SET TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
-</code></pre>
-</li>
-</ul>
-<pre><code>- **Support Flat folder same as Hive/Parquet**
-
-  This feature allows all carbondata and index files to keep directy under tablepath. Currently all carbondata/carbonindex files written under tablepath/Fact/Part0/Segment_NUM folder and it is not same as hive/parquet folder structure. This feature makes all files written will be directly under tablepath, it does not maintain any segment folder structure.This is useful for interoperability between the execution engines and plugin with other execution engines like hive or presto becomes easier.
-
-  Following table property enables this feature and default value is false.
-  ```
-   'flat_folder'='true'
-  ```
-  Example:
-  ```
-  CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES ('flat_folder'='true')
-  ```
-
-- **String longer than 32000 characters**
-
- In common scenarios, the length of string is less than 32000,
- so carbondata stores the length of content using Short to reduce memory and space consumption.
- To support string longer than 32000 characters, carbondata introduces a table property called `LONG_STRING_COLUMNS`.
- For these columns, carbondata internally stores the length of content using Integer.
-
- You can specify the columns as 'long string column' using below tblProperties:
-
- ```
- // specify col1, col2 as long string columns
- TBLPROPERTIES ('LONG_STRING_COLUMNS'='col1,col2')
- ```
-
- Besides, you can also use this property through DataFrame by
- ```
- df.format("carbondata")
-   .option("tableName", "carbonTable")
-   .option("long_string_columns", "col1, col2")
-   .save()
- ```
-
- If you are using Carbon-SDK, you can specify the datatype of long string column as `varchar`.
- You can refer to SDKwriterTestCase for example.
-
- **NOTE:** The LONG_STRING_COLUMNS can only be string/char/varchar columns and cannot be dictionary_include/sort_columns/complex columns.
-</code></pre>
-<h2>
-<a id="create-table-as-select" class="anchor" href="#create-table-as-select" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE AS SELECT</h2>
-<p>This function allows user 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.</p>
-<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name 
-STORED BY 'carbondata' 
-[TBLPROPERTIES (key1=val1, key2=val2, ...)] 
-AS select_statement;
-</code></pre>
-<h3>
-<a id="examples" class="anchor" href="#examples" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples</h3>
-<pre><code>carbon.sql("CREATE TABLE source_table(
-                           id INT,
-                           name STRING,
-                           city STRING,
-                           age INT)
-            STORED AS parquet")
-carbon.sql("INSERT INTO source_table SELECT 1,'bob','shenzhen',27")
-carbon.sql("INSERT INTO source_table SELECT 2,'david','shenzhen',31")
-
-carbon.sql("CREATE TABLE target_table
-            STORED BY 'carbondata'
-            AS SELECT city,avg(age) FROM source_table GROUP BY city")
-            
-carbon.sql("SELECT * FROM target_table").show
-  // results:
-  //    +--------+--------+
-  //    |    city|avg(age)|
-  //    +--------+--------+
-  //    |shenzhen|    29.0|
-  //    +--------+--------+
-
-</code></pre>
-<h2>
-<a id="create-external-table" class="anchor" href="#create-external-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE EXTERNAL TABLE</h2>
-<p>This function allows user to create external table by specifying location.</p>
-<pre><code>CREATE EXTERNAL TABLE [IF NOT EXISTS] [db_name.]table_name 
-STORED BY 'carbondata' LOCATION ?$FilesPath?
-</code></pre>
-<h3>
-<a id="create-external-table-on-managed-table-data-location" class="anchor" href="#create-external-table-on-managed-table-data-location" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create external table on managed table data location.</h3>
-<p>Managed table data location provided will have both FACT and Metadata folder.
-This data can be generated by creating a normal carbon table and use this path as $FilesPath in the above syntax.</p>
-<p><strong>Example:</strong></p>
-<pre><code>sql("CREATE TABLE origin(key INT, value STRING) STORED BY 'carbondata'")
-sql("INSERT INTO origin select 100,'spark'")
-sql("INSERT INTO origin select 200,'hive'")
-// creates a table in $storeLocation/origin
-
-sql(s"""
-|CREATE EXTERNAL TABLE source
-|STORED BY 'carbondata'
-|LOCATION '$storeLocation/origin'
-""".stripMargin)
-checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin"))
-</code></pre>
-<h3>
-<a id="create-external-table-on-non-transactional-table-data-location" class="anchor" href="#create-external-table-on-non-transactional-table-data-location" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create external table on Non-Transactional table data location.</h3>
-<p>Non-Transactional table data location will have only carbondata and carbonindex files, there will not be a metadata folder (table status and schema).
-Our SDK module currently support writing data in this format.</p>
-<p><strong>Example:</strong></p>
-<pre><code>sql(
-s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbondata' LOCATION
-|'$writerPath' """.stripMargin)
-</code></pre>
-<p>Here writer path will have carbondata and index files.
-This can be SDK output. Refer <a href="https://github.com/apache/carbondata/blob/master/docs/sdk-writer-guide.html" target=_blank>SDK Writer Guide</a>.</p>
-<p><strong>Note:</strong></p>
-<ol>
-<li>Dropping of the external table should not delete the files present in the location.</li>
-<li>When external table is created on non-transactional table data,
-external table will be registered with the schema of carbondata files.
-If multiple files with different schema is present, exception will be thrown.
-So, If table registered with one schema and files are of different schema,
-suggest to drop the external table and create again to register table with new schema.</li>
-</ol>
-<h2>
-<a id="create-database" class="anchor" href="#create-database" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE DATABASE</h2>
-<p>This function creates a new database. By default the database is created in Carbon store location, but you can also specify custom location.</p>
-<pre><code>CREATE DATABASE [IF NOT EXISTS] database_name [LOCATION path];
-</code></pre>
-<h3>
-<a id="example-1" class="anchor" href="#example-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example</h3>
-<pre><code>CREATE DATABASE carbon LOCATION ?hdfs://name_cluster/dir1/carbonstore?;
-</code></pre>
-<h2>
-<a id="table-management" class="anchor" href="#table-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>TABLE MANAGEMENT</h2>
-<h3>
-<a id="show-table" class="anchor" href="#show-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW TABLE</h3>
-<p>This command can be used to list all the tables in current database or all the tables of a specific database.</p>
-<pre><code>SHOW TABLES [IN db_Name]
-</code></pre>
-<p>Example:</p>
-<pre><code>SHOW TABLES
-OR
-SHOW TABLES IN defaultdb
-</code></pre>
-<h3>
-<a id="alter-table" class="anchor" href="#alter-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>ALTER TABLE</h3>
-<p>The following section introduce the commands to modify the physical or logical state of the existing table(s).</p>
-<ul>
-<li>
-<p><strong>RENAME TABLE</strong></p>
-<p>This command is used to rename the existing table.</p>
-<pre><code>ALTER TABLE [db_name.]table_name RENAME TO new_table_name
-</code></pre>
-<p>Examples:</p>
-<pre><code>ALTER TABLE carbon RENAME TO carbonTable
-OR
-ALTER TABLE test_db.carbon RENAME TO test_db.carbonTable
-</code></pre>
-</li>
-<li>
-<p><strong>ADD COLUMNS</strong></p>
-<p>This command is used to add a new column to the existing table.</p>
-<pre><code>ALTER TABLE [db_name.]table_name ADD COLUMNS (col_name data_type,...)
-TBLPROPERTIES('DICTIONARY_INCLUDE'='col_name,...',
-'DEFAULT.VALUE.COLUMN_NAME'='default_value')
-</code></pre>
-<p>Examples:</p>
-<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING)
-</code></pre>
-<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DICTIONARY_INCLUDE'='a1')
-</code></pre>
-<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DEFAULT.VALUE.a1'='10')
-</code></pre>
-<p>NOTE: Add Complex datatype columns is not supported.</p>
-</li>
-</ul>
-<p>Users can specify which columns to include and exclude for local dictionary generation after adding new columns. These will be appended with the already existing local dictionary include and exclude columns of main table respectively.</p>
-<pre><code>   ALTER TABLE carbon ADD COLUMNS (a1 STRING, b1 STRING) TBLPROPERTIES('LOCAL_DICTIONARY_INCLUDE'='a1','LOCAL_DICTIONARY_EXCLUDE'='b1')
-</code></pre>
-<ul>
-<li>
-<p><strong>DROP COLUMNS</strong></p>
-<p>This command is used to delete the existing column(s) in a table.</p>
-<pre><code>ALTER TABLE [db_name.]table_name DROP COLUMNS (col_name, ...)
-</code></pre>
-<p>Examples:</p>
-<pre><code>ALTER TABLE carbon DROP COLUMNS (b1)
-OR
-ALTER TABLE test_db.carbon DROP COLUMNS (b1)
-
-ALTER TABLE carbon DROP COLUMNS (c1,d1)
-</code></pre>
-<p>NOTE: Drop Complex child column is not supported.</p>
-</li>
-<li>
-<p><strong>CHANGE DATA TYPE</strong></p>
-<p>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.</p>
-<pre><code>ALTER TABLE [db_name.]table_name CHANGE col_name col_name changed_column_type
-</code></pre>
-<p>Valid Scenarios</p>
-<ul>
-<li>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.</li>
-<li>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.</li>
-<li>
-<strong>NOTE:</strong> The allowed range is 38,38 (precision, scale) and is a valid upper case scenario which is not resulting in data loss.</li>
-</ul>
-<p>Example1:Changing data type of column a1 from INT to BIGINT.</p>
-<pre><code>ALTER TABLE test_db.carbon CHANGE a1 a1 BIGINT
-</code></pre>
-<p>Example2:Changing decimal precision of column a1 from 10 to 18.</p>
-<pre><code>ALTER TABLE test_db.carbon CHANGE a1 a1 DECIMAL(18,2)
-</code></pre>
-</li>
-<li>
-<p><strong>MERGE INDEX</strong></p>
-<p>This command is used to merge all the CarbonData index files (.carbonindex) inside a segment to a single CarbonData index merge file (.carbonindexmerge). This enhances the first query performance.</p>
-<pre><code> ALTER TABLE [db_name.]table_name COMPACT 'SEGMENT_INDEX'
- ```
- 
- Examples:
- ```
- ALTER TABLE test_db.carbon COMPACT 'SEGMENT_INDEX'
- ```
- **NOTE:**
- * Merge index is not supported on streaming table.
- 
-</code></pre>
-</li>
-<li>
-<p><strong>SET and UNSET for Local Dictionary Properties</strong></p>
-<p>When set command is used, all the newly set properties will override the corresponding old properties if exists.</p>
-<p>Example to SET Local Dictionary Properties:</p>
-<pre><code>ALTER TABLE tablename SET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='false','LOCAL_DICTIONARY_THRESHOLD'='1000','LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
-</code></pre>
-<p>When Local Dictionary properties are unset, corresponding default values will be used for these properties.</p>
-<p>Example to UNSET Local Dictionary Properties:</p>
-<pre><code>ALTER TABLE tablename UNSET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE','LOCAL_DICTIONARY_THRESHOLD','LOCAL_DICTIONARY_INCLUDE','LOCAL_DICTIONARY_EXCLUDE')
-</code></pre>
-<p><strong>NOTE:</strong> For old tables, by default, local dictionary is disabled. If user wants local dictionary for these tables, user can enable/disable local dictionary for new data at their discretion.
-This can be achieved by using the alter table set command.</p>
-</li>
-</ul>
-<h3>
-<a id="drop-table" class="anchor" href="#drop-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DROP TABLE</h3>
-<p>This command is used to delete an existing table.</p>
-<pre><code>DROP TABLE [IF EXISTS] [db_name.]table_name
-</code></pre>
-<p>Example:</p>
-<pre><code>DROP TABLE IF EXISTS productSchema.productSalesTable
-</code></pre>
-<h3>
-<a id="refresh-table" class="anchor" href="#refresh-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>REFRESH TABLE</h3>
-<p>This command is used to register Carbon table to HIVE meta store catalogue from existing Carbon table data.</p>
-<pre><code>REFRESH TABLE $db_NAME.$table_NAME
-</code></pre>
-<p>Example:</p>
-<pre><code>REFRESH TABLE dbcarbon.productSalesTable
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>The new database name and the old database name should be same.</li>
-<li>Before executing this command the old table schema and data should be copied into the new database location.</li>
-<li>If the table is aggregate table, then all the aggregate tables should be copied to the new database location.</li>
-<li>For old store, the time zone of the source and destination cluster should be same.</li>
-<li>If old cluster used HIVE meta store to store schema, refresh will not work as schema file does not exist in file system.</li>
-</ul>
-<h3>
-<a id="table-and-column-comment" class="anchor" href="#table-and-column-comment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Table and Column Comment</h3>
-<p>You can provide more information on table by using table comment. Similarly you can provide more information about a particular column using column comment.
-You can see the column comment of an existing table using describe formatted command.</p>
-<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type [COMMENT col_comment], ...)]
-  [COMMENT table_comment]
-STORED BY 'carbondata'
-[TBLPROPERTIES (property_name=property_value, ...)]
-</code></pre>
-<p>Example:</p>
-<pre><code>CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
-                              productNumber Int COMMENT 'unique serial number for product')
-COMMENT ?This is table comment?
- STORED BY 'carbondata'
- TBLPROPERTIES ('DICTIONARY_INCLUDE'='productNumber')
-</code></pre>
-<p>You can also SET and UNSET table comment using ALTER command.</p>
-<p>Example to SET table comment:</p>
-<pre><code>ALTER TABLE carbon SET TBLPROPERTIES ('comment'='this table comment is modified');
-</code></pre>
-<p>Example to UNSET table comment:</p>
-<pre><code>ALTER TABLE carbon UNSET TBLPROPERTIES ('comment');
-</code></pre>
-<h2>
-<a id="load-data" class="anchor" href="#load-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LOAD DATA</h2>
-<h3>
-<a id="load-files-to-carbondata-table" class="anchor" href="#load-files-to-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LOAD FILES TO CARBONDATA TABLE</h3>
-<p>This command is used to load csv files to carbondata, OPTIONS are not mandatory for data loading process.
-Inside OPTIONS user can provide any options like DELIMITER, QUOTECHAR, FILEHEADER, ESCAPECHAR, MULTILINE as per requirement.</p>
-<pre><code>LOAD DATA [LOCAL] INPATH 'folder_path' 
-INTO TABLE [db_name.]table_name 
-OPTIONS(property_name=property_value, ...)
-</code></pre>
-<p>You can use the following options to load data:</p>
-<ul>
-<li>
-<p><strong>DELIMITER:</strong> Delimiters can be provided in the load command.</p>
-<pre><code>OPTIONS('DELIMITER'=',')
-</code></pre>
-</li>
-<li>
-<p><strong>QUOTECHAR:</strong> Quote Characters can be provided in the load command.</p>
-<pre><code>OPTIONS('QUOTECHAR'='"')
-</code></pre>
-</li>
-<li>
-<p><strong>COMMENTCHAR:</strong> Comment Characters can be provided in the load command if user want to comment lines.</p>
-<pre><code>OPTIONS('COMMENTCHAR'='#')
-</code></pre>
-</li>
-<li>
-<p><strong>HEADER:</strong> When you load the CSV file without the file header and the file header is the same with the table schema, then add 'HEADER'='false' to load data SQL as user need not provide the file header. By default the value is 'true'.
-false: CSV file is without file header.
-true: CSV file is with file header.</p>
-<pre><code>OPTIONS('HEADER'='false') 
-</code></pre>
-<p><strong>NOTE:</strong> If the HEADER option exist and is set to 'true', then the FILEHEADER option is not required.</p>
-</li>
-<li>
-<p><strong>FILEHEADER:</strong> Headers can be provided in the LOAD DATA command if headers are missing in the source files.</p>
-<pre><code>OPTIONS('FILEHEADER'='column1,column2') 
-</code></pre>
-</li>
-<li>
-<p><strong>MULTILINE:</strong> CSV with new line character in quotes.</p>
-<pre><code>OPTIONS('MULTILINE'='true') 
-</code></pre>
-</li>
-<li>
-<p><strong>ESCAPECHAR:</strong> Escape char can be provided if user want strict validation of escape character in CSV files.</p>
-<pre><code>OPTIONS('ESCAPECHAR'='\') 
-</code></pre>
-</li>
-<li>
-<p><strong>SKIP_EMPTY_LINE:</strong> This option will ignore the empty line in the CSV file during the data load.</p>
-<pre><code>OPTIONS('SKIP_EMPTY_LINE'='TRUE/FALSE') 
-</code></pre>
-</li>
-<li>
-<p><strong>COMPLEX_DELIMITER_LEVEL_1:</strong> Split the complex type data column in a row (eg., a$b$c --&gt; Array = {a,b,c}).</p>
-<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_1'='$') 
-</code></pre>
-</li>
-<li>
-<p><strong>COMPLEX_DELIMITER_LEVEL_2:</strong> Split the complex type nested data column in a row. Applies level_1 delimiter &amp; applies level_2 based on complex data type (eg., a:b$c:d --&gt; Array&gt; = {{a,b},{c,d}}).</p>
-<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_2'=':')
-</code></pre>
-</li>
-<li>
-<p><strong>ALL_DICTIONARY_PATH:</strong> All dictionary files path.</p>
-<pre><code>OPTIONS('ALL_DICTIONARY_PATH'='/opt/alldictionary/data.dictionary')
-</code></pre>
-</li>
-<li>
-<p><strong>COLUMNDICT:</strong> Dictionary file path for specified column.</p>
-<pre><code>OPTIONS('COLUMNDICT'='column1:dictionaryFilePath1,column2:dictionaryFilePath2')
-</code></pre>
-<p><strong>NOTE:</strong> ALL_DICTIONARY_PATH and COLUMNDICT can't be used together.</p>
-</li>
-<li>
-<p><strong>DATEFORMAT/TIMESTAMPFORMAT:</strong> Date and Timestamp format for specified column.</p>
-<pre><code>OPTIONS('DATEFORMAT' = 'yyyy-MM-dd','TIMESTAMPFORMAT'='yyyy-MM-dd HH:mm:ss')
-</code></pre>
-<p><strong>NOTE:</strong> Date formats are specified by date pattern strings. The date pattern letters in CarbonData are same as in JAVA. Refer to <a href="http://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html" target=_blank rel="nofollow">SimpleDateFormat</a>.</p>
-</li>
-<li>
-<p><strong>SORT COLUMN BOUNDS:</strong> Range bounds for sort columns.</p>
-<p>Suppose the table is created with 'SORT_COLUMNS'='name,id' and the range for name is aaa<del>zzz, the value range for id is 0</del>1000. Then during data loading, we can specify the following option to enhance data loading performance.</p>
-<pre><code>OPTIONS('SORT_COLUMN_BOUNDS'='f,250;l,500;r,750')
-</code></pre>
-<p>Each bound is separated by ';' and each field value in bound is separated by ','. In the example above, we provide 3 bounds to distribute records to 4 partitions. The values 'f','l','r' can evenly distribute the records. Inside carbondata, for a record we compare the value of sort columns with that of the bounds and decide which partition the record will be forwarded to.</p>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>SORT_COLUMN_BOUNDS will be used only when the SORT_SCOPE is 'local_sort'.</li>
-<li>Carbondata will use these bounds as ranges to process data concurrently during the final sort percedure. The records will be sorted and written out inside each partition. Since the partition is sorted, all records will be sorted.</li>
-<li>Since the actual order and literal order of the dictionary column are not necessarily the same, we do not recommend you to use this feature if the first sort column is 'dictionary_include'.</li>
-<li>The option works better if your CPU usage during loading is low. If your system is already CPU tense, better not to use this option. Besides, it depends on the user to specify the bounds. If user does not know the exactly bounds to make the data distributed evenly among the bounds, loading performance will still be better than before or at least the same as before.</li>
-<li>Users can find more information about this option in the description of PR1953.</li>
-</ul>
-</li>
-<li>
-<p><strong>SINGLE_PASS:</strong> 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.</p>
-</li>
-</ul>
-<p>This option specifies whether to use single pass for loading data or not. By default this option is set to FALSE.</p>
-<pre><code> OPTIONS('SINGLE_PASS'='TRUE')
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>If this option is set to TRUE then data loading will take less time.</li>
-<li>If this option is set to some invalid value other than TRUE or FALSE then it uses the default value.</li>
-</ul>
-<p>Example:</p>
-<pre><code>LOAD DATA local inpath '/opt/rawdata/data.csv' INTO table carbontable
-options('DELIMITER'=',', 'QUOTECHAR'='"','COMMENTCHAR'='#',
-'HEADER'='false',
-'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')
-</code></pre>
-<ul>
-<li>
-<p><strong>BAD RECORDS HANDLING:</strong> Methods of handling bad records are as follows:</p>
-<ul>
-<li>Load all of the data before dealing with the errors.</li>
-<li>Clean or delete bad records before loading data or stop the loading when bad records are found.</li>
-</ul>
-<pre><code>OPTIONS('BAD_RECORDS_LOGGER_ENABLE'='true', 'BAD_RECORD_PATH'='hdfs://hacluster/tmp/carbon', 'BAD_RECORDS_ACTION'='REDIRECT', 'IS_EMPTY_DATA_BAD_RECORD'='false')
-</code></pre>
-</li>
-</ul>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>BAD_RECORDS_ACTION property can have four type of actions for bad records FORCE, REDIRECT, IGNORE and FAIL.</li>
-<li>FAIL option is its Default value. If the FAIL option is used, then data loading fails if any bad records are found.</li>
-<li>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.</li>
-<li>If the FORCE option is used, then it auto-converts the data by storing the bad records as NULL before Loading data.</li>
-<li>If the IGNORE option is used, then bad records are neither loaded nor written to the separate CSV file.</li>
-<li>In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails.</li>
-<li>The default maximum number of characters per column is 32000. If there are more than 32000 characters in a column, please refer to <em>String longer than 32000 characters</em> section.</li>
-<li>Since Bad Records Path can be specified in create, load and carbon properties.
-Therefore, value specified in load will have the highest priority, and value specified in carbon properties will have the least priority.</li>
-</ul>
-<p><strong>Bad Records Path:</strong></p>
-<p>This property is used to specify the location where bad records would be written.</p>
-<pre><code>TBLPROPERTIES('BAD_RECORDS_PATH'='/opt/badrecords'')
-</code></pre>
-<p>Example:</p>
-<pre><code>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')
-</code></pre>
-<ul>
-<li>
-<strong>GLOBAL_SORT_PARTITIONS:</strong> If the SORT_SCOPE is defined as GLOBAL_SORT, then user can specify the number of partitions to use while shuffling data for sort using GLOBAL_SORT_PARTITIONS. If it is not configured, or configured less than 1, then it uses the number of map task as reduce task. It is recommended that each reduce task deal with 512MB-1GB data.</li>
-</ul>
-<pre><code>OPTIONS('GLOBAL_SORT_PARTITIONS'='2')
-</code></pre>
-<p>NOTE:</p>
-<ul>
-<li>GLOBAL_SORT_PARTITIONS should be Integer type, the range is [1,Integer.MaxValue].</li>
-<li>It is only used when the SORT_SCOPE is GLOBAL_SORT.</li>
-</ul>
-<h3>
-<a id="insert-data-into-carbondata-table" class="anchor" href="#insert-data-into-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>INSERT DATA INTO CARBONDATA TABLE</h3>
-<p>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.</p>
-<pre><code>INSERT INTO TABLE &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
-[ WHERE { &lt;filter_condition&gt; } ]
-</code></pre>
-<p>You can also omit the <code>table</code> keyword and write your query as:</p>
-<pre><code>INSERT INTO &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
-[ WHERE { &lt;filter_condition&gt; } ]
-</code></pre>
-<p>Overwrite insert data:</p>
-<pre><code>INSERT OVERWRITE TABLE &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
-[ WHERE { &lt;filter_condition&gt; } ]
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>The source table and the CarbonData table must have the same table schema.</li>
-<li>The data type of source and destination table columns should be same</li>
-<li>INSERT INTO command does not support partial success if bad records are found, it will fail.</li>
-<li>Data cannot be loaded or updated in source table while insert from source table to target table is in progress.</li>
-</ul>
-<p>Examples</p>
-<pre><code>INSERT INTO table1 SELECT item1, sum(item2 + 1000) as result FROM table2 group by item1
-</code></pre>
-<pre><code>INSERT INTO table1 SELECT item1, item2, item3 FROM table2 where item2='xyz'
-</code></pre>
-<pre><code>INSERT OVERWRITE TABLE table1 SELECT * FROM TABLE2
-</code></pre>
-<h2>
-<a id="update-and-delete" class="anchor" href="#update-and-delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>UPDATE AND DELETE</h2>
-<h3>
-<a id="update" class="anchor" href="#update" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>UPDATE</h3>
-<p>This command will allow to update the CarbonData table based on the column expression and optional filter conditions.</p>
-<pre><code>UPDATE &lt;table_name&gt; 
-SET (column_name1, column_name2, ... column_name n) = (column1_expression , column2_expression, ... column n_expression )
-[ WHERE { &lt;filter_condition&gt; } ]
-</code></pre>
-<p>alternatively the following command can also be used for updating the CarbonData Table :</p>
-<pre><code>UPDATE &lt;table_name&gt;
-SET (column_name1, column_name2) =(select sourceColumn1, sourceColumn2 from sourceTable [ WHERE { &lt;filter_condition&gt; } ] )
-[ WHERE { &lt;filter_condition&gt; } ]
-</code></pre>
-<p><strong>NOTE:</strong> The update command fails if multiple input rows in source table are matched with single row in destination table.</p>
-<p>Examples:</p>
-<pre><code>UPDATE t3 SET (t3_salary) = (t3_salary + 9) WHERE t3_name = 'aaa1'
-</code></pre>
-<pre><code>UPDATE t3 SET (t3_date, t3_country) = ('2017-11-18', 'india') WHERE t3_salary &lt; 15003
-</code></pre>
-<pre><code>UPDATE t3 SET (t3_country, t3_name) = (SELECT t5_country, t5_name FROM t5 WHERE t5_id = 5) WHERE t3_id &lt; 5
-</code></pre>
-<pre><code>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 &lt; 5
-</code></pre>
-<pre><code>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 &gt;6
-</code></pre>
-<p>NOTE: Update Complex datatype columns is not supported.</p>
-<h3>
-<a id="delete" class="anchor" href="#delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE</h3>
-<p>This command allows us to delete records from CarbonData table.</p>
-<pre><code>DELETE FROM table_name [WHERE expression]
-</code></pre>
-<p>Examples:</p>
-<pre><code>DELETE FROM carbontable WHERE column1  = 'china'
-</code></pre>
-<pre><code>DELETE FROM carbontable WHERE column1 IN ('china', 'USA')
-</code></pre>
-<pre><code>DELETE FROM carbontable WHERE column1 IN (SELECT column11 FROM sourceTable2)
-</code></pre>
-<pre><code>DELETE FROM carbontable WHERE column1 IN (SELECT column11 FROM sourceTable2 WHERE column1 = 'USA')
-</code></pre>
-<h2>
-<a id="compaction" class="anchor" href="#compaction" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COMPACTION</h2>
-<p>Compaction improves the query performance significantly.</p>
-<p>There are several types of compaction.</p>
-<pre><code>ALTER TABLE [db_name.]table_name COMPACT 'MINOR/MAJOR/CUSTOM'
-</code></pre>
-<ul>
-<li><strong>Minor Compaction</strong></li>
-</ul>
-<p>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:</p>
-<ul>
-<li>Level 1: Merging of the segments which are not yet compacted.</li>
-<li>Level 2: Merging of the compacted segments again to form a larger segment.</li>
-</ul>
-<pre><code>ALTER TABLE table_name COMPACT 'MINOR'
-</code></pre>
-<ul>
-<li><strong>Major Compaction</strong></li>
-</ul>
-<p>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.
-Configure the property carbon.major.compaction.size with appropriate value in MB.</p>
-<p>This command merges the specified number of segments into one segment:</p>
-<pre><code>ALTER TABLE table_name COMPACT 'MAJOR'
-</code></pre>
-<ul>
-<li><strong>Custom Compaction</strong></li>
-</ul>
-<p>In Custom compaction, user can directly specify segment ids to be merged into one large segment.
-All specified segment ids should exist and be valid, otherwise compaction will fail.
-Custom compaction is usually done during the off-peak time.</p>
-<pre><code>ALTER TABLE table_name COMPACT 'CUSTOM' WHERE SEGMENT.ID IN (2,3,4)
-</code></pre>
-<p>NOTE: Compaction is unsupported for table containing Complex columns.</p>
-<ul>
-<li><strong>CLEAN SEGMENTS AFTER Compaction</strong></li>
-</ul>
-<p>Clean the segments which are compacted:</p>
-<pre><code>CLEAN FILES FOR TABLE carbon_table
-</code></pre>
-<h2>
-<a id="partition" class="anchor" href="#partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>PARTITION</h2>
-<h3>
-<a id="standard-partition" class="anchor" href="#standard-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>STANDARD PARTITION</h3>
-<p>The partition is similar as spark and hive partition, user can use any column to build partition:</p>
-<h4>
-<a id="create-partition-table" class="anchor" href="#create-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Partition Table</h4>
-<p>This command allows you to create table with partition.</p>
-<pre><code>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, ...)]
-</code></pre>
-<p>Example:</p>
-<pre><code> 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'
-</code></pre>
-<p>NOTE: Hive partition is not supported on complex datatype columns.</p>
-<h4>
-<a id="load-data-using-static-partition" class="anchor" href="#load-data-using-static-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Load Data Using Static Partition</h4>
-<p>This command allows you to load data using static partition.</p>
-<pre><code>LOAD DATA [LOCAL] INPATH 'folder_path' 
-INTO TABLE [db_name.]table_name PARTITION (partition_spec) 
-OPTIONS(property_name=property_value, ...)    
-INSERT INTO INTO TABLE [db_name.]table_name PARTITION (partition_spec) &lt;SELECT STATEMENT&gt;
-</code></pre>
-<p>Example:</p>
-<pre><code>LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.csv'
-INTO TABLE locationTable
-PARTITION (country = 'US', state = 'CA')  
-INSERT INTO TABLE locationTable
-PARTITION (country = 'US', state = 'AL')
-SELECT &lt;columns list excluding partition columns&gt; FROM another_user
-</code></pre>
-<h4>
-<a id="load-data-using-dynamic-partition" class="anchor" href="#load-data-using-dynamic-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Load Data Using Dynamic Partition</h4>
-<p>This command allows you to load data using dynamic partition. If partition spec is not specified, then the partition is considered as dynamic.</p>
-<p>Example:</p>
-<pre><code>LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.csv'
-INTO TABLE locationTable          
-INSERT INTO TABLE locationTable
-SELECT &lt;columns list excluding partition columns&gt; FROM another_user
-</code></pre>
-<h4>
-<a id="show-partitions" class="anchor" href="#show-partitions" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Show Partitions</h4>
-<p>This command gets the Hive partition information of the table</p>
-<pre><code>SHOW PARTITIONS [db_name.]table_name
-</code></pre>
-<h4>
-<a id="drop-partition" class="anchor" href="#drop-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Drop Partition</h4>
-<p>This command drops the specified Hive partition only.</p>
-<pre><code>ALTER TABLE table_name DROP [IF EXISTS] PARTITION (part_spec, ...)
-</code></pre>
-<p>Example:</p>
-<pre><code>ALTER TABLE locationTable DROP PARTITION (country = 'US');
-</code></pre>
-<h4>
-<a id="insert-overwrite" class="anchor" href="#insert-overwrite" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Insert OVERWRITE</h4>
-<p>This command allows you to insert or load overwrite on a specific partition.</p>
-<pre><code> INSERT OVERWRITE TABLE table_name
- PARTITION (column = 'partition_name')
- select_statement
-</code></pre>
-<p>Example:</p>
-<pre><code>INSERT OVERWRITE TABLE partitioned_user
-PARTITION (country = 'US')
-SELECT * FROM another_user au 
-WHERE au.country = 'US';
-</code></pre>
-<h3>
-<a id="carbondata-partitionhashrangelist----alpha-feature-this-partition-feature-does-not-support-update-and-delete-data" class="anchor" href="#carbondata-partitionhashrangelist----alpha-feature-this-partition-feature-does-not-support-update-and-delete-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CARBONDATA PARTITION(HASH,RANGE,LIST) -- Alpha feature, this partition feature does not support update and delete data.</h3>
-<p>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.</p>
-<h3>
-<a id="create-hash-partition-table" class="anchor" href="#create-hash-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Hash Partition Table</h3>
-<p>This command allows us to create hash partition.</p>
-<pre><code>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' ...)]
-</code></pre>
-<p><strong>NOTE:</strong> N is the number of hash partitions</p>
-<p>Example:</p>
-<pre><code>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')
-</code></pre>
-<h3>
-<a id="create-range-partition-table" class="anchor" href="#create-range-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Range Partition Table</h3>
-<p>This command allows us to create range partition.</p>
-<pre><code>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, ...')]
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>The 'RANGE_INFO' must be defined in ascending order in the table properties.</li>
-<li>The default format for partition column of Date/Timestamp type is yyyy-MM-dd. Alternate formats for Date/Timestamp could be defined in CarbonProperties.</li>
-</ul>
-<p>Example:</p>
-<pre><code>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')
-</code></pre>
-<h3>
-<a id="create-list-partition-table" class="anchor" href="#create-list-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create List Partition Table</h3>
-<p>This command allows us to create list partition.</p>
-<pre><code>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, ...')]
-</code></pre>
-<p><strong>NOTE:</strong> List partition supports list info in one level group.</p>
-<p>Example:</p>
-<pre><code>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')
-</code></pre>
-<h3>
-<a id="show-partitions-1" class="anchor" href="#show-partitions-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Show Partitions</h3>
-<p>The following command is executed to get the partition information of the table</p>
-<pre><code>SHOW PARTITIONS [db_name.]table_name
-</code></pre>
-<h3>
-<a id="add-a-new-partition" class="anchor" href="#add-a-new-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Add a new partition</h3>
-<pre><code>ALTER TABLE [db_name].table_name ADD PARTITION('new_partition')
-</code></pre>
-<h3>
-<a id="split-a-partition" class="anchor" href="#split-a-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Split a partition</h3>
-<pre><code>ALTER TABLE [db_name].table_name SPLIT PARTITION(partition_id) INTO('new_partition1', 'new_partition2'...)
-</code></pre>
-<h3>
-<a id="drop-a-partition" class="anchor" href="#drop-a-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Drop a partition</h3>
-<p>Only drop partition definition, but keep data</p>
-<pre><code>  ALTER TABLE [db_name].table_name DROP PARTITION(partition_id)
-</code></pre>
-<p>Drop both partition definition and data</p>
-<pre><code>ALTER TABLE [db_name].table_name DROP PARTITION(partition_id) WITH DATA
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>Hash partition table is not supported for ADD, SPLIT and DROP commands.</li>
-<li>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.</li>
-</ul>
-<pre><code>SegmentDir/0_batchno0-0-1502703086921.carbonindex
-          ^
-SegmentDir/part-0-0_batchno0-0-1502703086921.carbondata
-                   ^
-</code></pre>
-<p>Here are some useful tips to improve query performance of carbonData partition table:</p>
-<ul>
-<li>The partitioned column can be excluded from SORT_COLUMNS, this will let other columns to do the efficient sorting.</li>
-<li>When writing SQL on a partition table, try to use filters on the partition column.</li>
-</ul>
-<h2>
-<a id="bucketing" class="anchor" href="#bucketing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>BUCKETING</h2>
-<p>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.</p>
-<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
-                  [(col_name data_type, ...)]
-STORED BY 'carbondata'
-TBLPROPERTIES('BUCKETNUMBER'='noOfBuckets',
-'BUCKETCOLUMNS'='columnname')
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>Bucketing cannot be performed for columns of Complex Data Types.</li>
-<li>Columns in the BUCKETCOLUMN parameter must be dimensions. The BUCKETCOLUMN parameter cannot be a measure or a combination of measures and dimensions.</li>
-</ul>
-<p>Example:</p>
-<pre><code>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')
-</code></pre>
-<h2>
-<a id="segment-management" class="anchor" href="#segment-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SEGMENT MANAGEMENT</h2>
-<h3>
-<a id="show-segment" class="anchor" href="#show-segment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW SEGMENT</h3>
-<p>This command is used to list the segments of CarbonData table.</p>
-<pre><code>SHOW [HISTORY] SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
-</code></pre>
-<p>Example:
-Show visible segments</p>
-<pre><code>SHOW SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4
-</code></pre>
-<p>Show all segments, include invisible segments</p>
-<pre><code>SHOW HISTORY SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4
-</code></pre>
-<h3>
-<a id="delete-segment-by-id" class="anchor" href="#delete-segment-by-id" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY ID</h3>
-<p>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.</p>
-<p>The following command will get the segmentID.</p>
-<pre><code>SHOW SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
-</code></pre>
-<p>After you retrieve the segment ID of the segment that you want to delete, execute the following command to delete the selected segment.</p>
-<pre><code>DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.ID IN (segment_id1, segments_id2, ...)
-</code></pre>
-<p>Example:</p>
-<pre><code>DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0)
-DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0,5,8)
-</code></pre>
-<h3>
-<a id="delete-segment-by-date" class="anchor" href="#delete-segment-by-date" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY DATE</h3>
-<p>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.</p>
-<pre><code>DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.STARTTIME BEFORE DATE_VALUE
-</code></pre>
-<p>Example:</p>
-<pre><code>DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.STARTTIME BEFORE '2017-06-01 12:05:06' 
-</code></pre>
-<h3>
-<a id="query-data-with-specified-segments" class="anchor" href="#query-data-with-specified-segments" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>QUERY DATA WITH SPECIFIED SEGMENTS</h3>
-<p>This command is used to read data from specified segments during CarbonScan.</p>
-<p>Get the Segment ID:</p>
-<pre><code>SHOW SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
-</code></pre>
-<p>Set the segment IDs for table</p>
-<pre><code>SET carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt; = &lt;list of segment IDs&gt;
-</code></pre>
-<p><strong>NOTE:</strong>
-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.</p>
-<p>If user wants to query with segments reading in multi threading mode, then CarbonSession. threadSet can be used instead of SET query.</p>
-<pre><code>CarbonSession.threadSet ("carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt;","&lt;list of segment IDs&gt;");
-</code></pre>
-<p>Reset the segment IDs</p>
-<pre><code>SET carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt; = *;
-</code></pre>
-<p>If user wants to query with segments reading in multi threading mode, then CarbonSession. threadSet can be used instead of SET query.</p>
-<pre><code>CarbonSession.threadSet ("carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt;","*");
-</code></pre>
-<p><strong>Examples:</strong></p>
-<ul>
-<li>Example to show the list of segment IDs,segment status, and other required details and then specify the list of segments to be read.</li>
-</ul>
-<pre><code>SHOW SEGMENTS FOR carbontable1;
-
-SET carbon.input.segments.db.carbontable1 = 1,3,9;
-</code></pre>
-<ul>
-<li>Example to query with segments reading in multi threading mode:</li>
-</ul>
-<pre><code>CarbonSession.threadSet ("carbon.input.segments.db.carbontable_Multi_Thread","1,3");
-</code></pre>
-<ul>
-<li>Example for threadset in multithread environment (following shows how it is used in Scala code):</li>
-</ul>
-<pre><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();
-   }
- }
-</code></pre>
-</div>
-</div>
-</div>
-</div>
-<div class="doc-footer">
-    <a href="#top" class="scroll-top">Top</a>
-</div>
-</div>
-</section>
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/datamap-developer-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/datamap-developer-guide.html b/src/main/webapp/datamap-developer-guide.html
index 073af96..4b9aa4b 100644
--- a/src/main/webapp/datamap-developer-guide.html
+++ b/src/main/webapp/datamap-developer-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -240,8 +240,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -256,4 +255,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/datamap-management.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/datamap-management.html b/src/main/webapp/datamap-management.html
index 8727594..e2e89f3 100644
--- a/src/main/webapp/datamap-management.html
+++ b/src/main/webapp/datamap-management.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -368,8 +368,7 @@ $(function() {
     $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -384,4 +383,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file


[13/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/sdk-guide.html
----------------------------------------------------------------------
diff --git a/content/sdk-guide.html b/content/sdk-guide.html
index cc55dc0..95ae214 100644
--- a/content/sdk-guide.html
+++ b/content/sdk-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -502,6 +553,22 @@ public CarbonWriterBuilder taskNo(long taskNo);
 public CarbonWriterBuilder withLoadOptions(Map&lt;String, String&gt; options);
 </code></pre>
 <pre><code>/**
+ * To support the table properties for sdk writer
+ *
+ * @param options key,value pair of create table properties.
+ * supported keys values are
+ * a. blocksize -- [1-2048] values in MB. Default value is 1024
+ * b. blockletsize -- values in MB. Default value is 64 MB
+ * c. localDictionaryThreshold -- positive value, default is 10000
+ * d. enableLocalDictionary -- true / false. Default is false
+ * e. sortcolumns -- comma separated column. "c1,c2". Default all dimensions are sorted.
+ *
+ * @return updated CarbonWriterBuilder
+ */
+public CarbonWriterBuilder withTableProperties(Map&lt;String, String&gt; options);
+</code></pre>
+<pre><code>/**
+* this writer is not thread safe, use buildThreadSafeWriterForCSVInput in multi thread environment
 * Build a {@link CarbonWriter}, which accepts row in CSV format object
 * @param schema carbon Schema object {org.apache.carbondata.sdk.file.Schema}
 * @return CSVCarbonWriter
@@ -511,6 +578,19 @@ public CarbonWriterBuilder withLoadOptions(Map&lt;String, String&gt; options);
 public CarbonWriter buildWriterForCSVInput(org.apache.carbondata.sdk.file.Schema schema) throws IOException, InvalidLoadOptionException;
 </code></pre>
 <pre><code>/**
+* Can use this writer in multi-thread instance.
+* Build a {@link CarbonWriter}, which accepts row in CSV format
+* @param schema carbon Schema object {org.apache.carbondata.sdk.file.Schema}
+* @param numOfThreads number of threads() in which .write will be called.              
+* @return CSVCarbonWriter
+* @throws IOException
+* @throws InvalidLoadOptionException
+*/
+public CarbonWriter buildThreadSafeWriterForCSVInput(Schema schema, short numOfThreads)
+  throws IOException, InvalidLoadOptionException;
+</code></pre>
+<pre><code>/**
+* this writer is not thread safe, use buildThreadSafeWriterForAvroInput in multi thread environment
 * Build a {@link CarbonWriter}, which accepts Avro format object
 * @param avroSchema avro Schema object {org.apache.avro.Schema}
 * @return AvroCarbonWriter 
@@ -520,6 +600,19 @@ public CarbonWriter buildWriterForCSVInput(org.apache.carbondata.sdk.file.Schema
 public CarbonWriter buildWriterForAvroInput(org.apache.avro.Schema schema) throws IOException, InvalidLoadOptionException;
 </code></pre>
 <pre><code>/**
+* Can use this writer in multi-thread instance.
+* Build a {@link CarbonWriter}, which accepts Avro object
+* @param avroSchema avro Schema object {org.apache.avro.Schema}
+* @param numOfThreads number of threads() in which .write will be called.
+* @return AvroCarbonWriter
+* @throws IOException
+* @throws InvalidLoadOptionException
+*/
+public CarbonWriter buildThreadSafeWriterForAvroInput(org.apache.avro.Schema avroSchema, short numOfThreads)
+  throws IOException, InvalidLoadOptionException
+</code></pre>
+<pre><code>/**
+* this writer is not thread safe, use buildThreadSafeWriterForJsonInput in multi thread environment
 * Build a {@link CarbonWriter}, which accepts Json object
 * @param carbonSchema carbon Schema object
 * @return JsonCarbonWriter
@@ -528,6 +621,17 @@ public CarbonWriter buildWriterForAvroInput(org.apache.avro.Schema schema) throw
 */
 public JsonCarbonWriter buildWriterForJsonInput(Schema carbonSchema);
 </code></pre>
+<pre><code>/**
+* Can use this writer in multi-thread instance.
+* Build a {@link CarbonWriter}, which accepts Json object
+* @param carbonSchema carbon Schema object
+* @param numOfThreads number of threads() in which .write will be called.
+* @return JsonCarbonWriter
+* @throws IOException
+* @throws InvalidLoadOptionException
+*/
+public JsonCarbonWriter buildThreadSafeWriterForJsonInput(Schema carbonSchema, short numOfThreads)
+</code></pre>
 <h3>
 <a id="class-orgapachecarbondatasdkfilecarbonwriter" class="anchor" href="#class-orgapachecarbondatasdkfilecarbonwriter" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.CarbonWriter</h3>
 <pre><code>/**
@@ -536,7 +640,7 @@ public JsonCarbonWriter buildWriterForJsonInput(Schema carbonSchema);
 *                      which is one row of data.
 * If CSVCarbonWriter, object is of type String[], which is one row of data
 * If JsonCarbonWriter, object is of type String, which is one row of json
-* Note: This API is not thread safe
+* Note: This API is not thread safe if writer is not built with number of threads argument.
 * @param object
 * @throws IOException
 */
@@ -780,7 +884,6 @@ External client can make use of this reader to read CarbonData files without Car
    *
    * @param dataFilePath complete path including carbondata file name
    * @return Schema object
-   * @throws IOException
    */
   public static Schema readSchemaInDataFile(String dataFilePath);
 </code></pre>
@@ -882,7 +985,11 @@ public String getProperty(String key);
 */
 public String getProperty(String key, String defaultValue);
 </code></pre>
-<p>Reference : <a href="http://carbondata.apache.org/configuration-parameters.html" target=_blank rel="nofollow">list of carbon properties</a></p>
+<p>Reference : <a href="./configuration-parameters.html">list of carbon properties</a></p>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__api').addClass('selected'); });
+</script>
 </div>
 </div>
 </div>
@@ -898,4 +1005,4 @@ public String getProperty(String key, String defaultValue);
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/sdk-writer-guide.html
----------------------------------------------------------------------
diff --git a/content/sdk-writer-guide.html b/content/sdk-writer-guide.html
index 36bb9ad..5553194 100644
--- a/content/sdk-writer-guide.html
+++ b/content/sdk-writer-guide.html
@@ -546,4 +546,4 @@ public static org.apache.carbondata.sdk.file.Schema getCarbonSchemaFromAvroSchem
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/security.html
----------------------------------------------------------------------
diff --git a/content/security.html b/content/security.html
index 9168fa3..2aa4b8e 100644
--- a/content/security.html
+++ b/content/security.html
@@ -205,4 +205,4 @@
 <script src="js/custom.js"></script>
 <script src="js/mdNavigation.js" type="text/javascript"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/segment-management-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/segment-management-on-carbondata.html b/content/segment-management-on-carbondata.html
new file mode 100644
index 0000000..528ee9d
--- /dev/null
+++ b/content/segment-management-on-carbondata.html
@@ -0,0 +1,326 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h2>
+<a id="segment-management" class="anchor" href="#segment-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SEGMENT MANAGEMENT</h2>
+<p>Each load into CarbonData is written into a separate folder called Segment.Segments is a powerful
+concept which helps to maintain consistency of data and easy transaction management.CarbonData provides DML (Data Manipulation Language) commands to maintain the segments.</p>
+<ul>
+<li><a href="#show-segment">Show Segments</a></li>
+<li><a href="#delete-segment-by-id">Delete Segment by ID</a></li>
+<li><a href="#delete-segment-by-date">Delete Segment by Date</a></li>
+<li><a href="#query-data-with-specified-segments">Query Data with Specified Segments</a></li>
+</ul>
+<h3>
+<a id="show-segment" class="anchor" href="#show-segment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW SEGMENT</h3>
+<p>This command is used to list the segments of CarbonData table.</p>
+<pre><code>SHOW [HISTORY] SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
+</code></pre>
+<p>Example:
+Show visible segments</p>
+<pre><code>SHOW SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4
+</code></pre>
+<p>Show all segments, include invisible segments</p>
+<pre><code>SHOW HISTORY SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4
+</code></pre>
+<h3>
+<a id="delete-segment-by-id" class="anchor" href="#delete-segment-by-id" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY ID</h3>
+<p>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.</p>
+<p>The following command will get the segmentID.</p>
+<pre><code>SHOW SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
+</code></pre>
+<p>After you retrieve the segment ID of the segment that you want to delete, execute the following command to delete the selected segment.</p>
+<pre><code>DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.ID IN (segment_id1, segments_id2, ...)
+</code></pre>
+<p>Example:</p>
+<pre><code>DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0)
+DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0,5,8)
+</code></pre>
+<h3>
+<a id="delete-segment-by-date" class="anchor" href="#delete-segment-by-date" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY DATE</h3>
+<p>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.</p>
+<pre><code>DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.STARTTIME BEFORE DATE_VALUE
+</code></pre>
+<p>Example:</p>
+<pre><code>DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.STARTTIME BEFORE '2017-06-01 12:05:06' 
+</code></pre>
+<h3>
+<a id="query-data-with-specified-segments" class="anchor" href="#query-data-with-specified-segments" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>QUERY DATA WITH SPECIFIED SEGMENTS</h3>
+<p>This command is used to read data from specified segments during CarbonScan.</p>
+<p>Get the Segment ID:</p>
+<pre><code>SHOW SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
+</code></pre>
+<p>Set the segment IDs for table</p>
+<pre><code>SET carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt; = &lt;list of segment IDs&gt;
+</code></pre>
+<p><strong>NOTE:</strong>
+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.</p>
+<p>If user wants to query with segments reading in multi threading mode, then CarbonSession. threadSet can be used instead of SET query.</p>
+<pre><code>CarbonSession.threadSet ("carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt;","&lt;list of segment IDs&gt;");
+</code></pre>
+<p>Reset the segment IDs</p>
+<pre><code>SET carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt; = *;
+</code></pre>
+<p>If user wants to query with segments reading in multi threading mode, then CarbonSession. threadSet can be used instead of SET query.</p>
+<pre><code>CarbonSession.threadSet ("carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt;","*");
+</code></pre>
+<p><strong>Examples:</strong></p>
+<ul>
+<li>Example to show the list of segment IDs,segment status, and other required details and then specify the list of segments to be read.</li>
+</ul>
+<pre><code>SHOW SEGMENTS FOR carbontable1;
+
+SET carbon.input.segments.db.carbontable1 = 1,3,9;
+</code></pre>
+<ul>
+<li>Example to query with segments reading in multi threading mode:</li>
+</ul>
+<pre><code>CarbonSession.threadSet ("carbon.input.segments.db.carbontable_Multi_Thread","1,3");
+</code></pre>
+<ul>
+<li>Example for threadset in multithread environment (following shows how it is used in Scala code):</li>
+</ul>
+<pre><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();
+   }
+ }
+</code></pre>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/streaming-guide.html
----------------------------------------------------------------------
diff --git a/content/streaming-guide.html b/content/streaming-guide.html
index 6e2e7ee..c6d8391 100644
--- a/content/streaming-guide.html
+++ b/content/streaming-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -432,6 +483,17 @@ streaming table using following DDL.</p>
 <li>block delete "streaming" segment while the streaming ingestion is running.</li>
 <li>block drop the streaming table while the streaming ingestion is running.</li>
 </ol>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -447,4 +509,4 @@ streaming table using following DDL.</p>
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/supported-data-types-in-carbondata.html
----------------------------------------------------------------------
diff --git a/content/supported-data-types-in-carbondata.html b/content/supported-data-types-in-carbondata.html
index 780cab0..e0ad647 100644
--- a/content/supported-data-types-in-carbondata.html
+++ b/content/supported-data-types-in-carbondata.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -189,7 +240,7 @@
 <li>VARCHAR</li>
 </ul>
 <p><strong>NOTE</strong>: For string longer than 32000 characters, use <code>LONG_STRING_COLUMNS</code> in table property.
-Please refer to TBLProperties in <a href="https://github.com/apache/carbondata/blob/master/docs/data-management-on-carbondata.html#create-table">CreateTable</a> for more information.</p>
+Please refer to TBLProperties in <a href="./ddl-of-carbondata.html#create-table">CreateTable</a> for more information.</p>
 </li>
 <li>
 <p>Complex Types</p>
@@ -208,6 +259,17 @@ Please refer to TBLProperties in <a href="https://github.com/apache/carbondata/b
 </ul>
 </li>
 </ul>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -223,4 +285,4 @@ Please refer to TBLProperties in <a href="https://github.com/apache/carbondata/b
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/timeseries-datamap-guide.html
----------------------------------------------------------------------
diff --git a/content/timeseries-datamap-guide.html b/content/timeseries-datamap-guide.html
index 548be2e..497b02f 100644
--- a/content/timeseries-datamap-guide.html
+++ b/content/timeseries-datamap-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -168,16 +219,16 @@
 </ul>
 <h2>
 <a id="timeseries-datamap-introduction-alpha-feature" class="anchor" href="#timeseries-datamap-introduction-alpha-feature" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Timeseries DataMap Introduction (Alpha Feature)</h2>
-<p>Timeseries DataMap a pre-aggregate table implementation based on 'pre-aggregate' DataMap.
+<p>Timeseries DataMap is a pre-aggregate table implementation based on 'pre-aggregate' DataMap.
 Difference is that Timeseries DataMap has built-in understanding of time hierarchy and
 levels: year, month, day, hour, minute, so that it supports automatic roll-up in time dimension
 for query.</p>
+<p><strong>CAUTION:</strong> Current version of CarbonData does not support roll-up.It will be implemented in future versions.</p>
 <p>The data loading, querying, compaction command and its behavior is the same as preaggregate DataMap.
-Please refer to <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/preaggregate-datamap-guide.html" target=_blank>Pre-aggregate DataMap</a>
+Please refer to <a href="./preaggregate-datamap-guide.html">Pre-aggregate DataMap</a>
 for more information.</p>
 <p>To use this datamap, user can create multiple timeseries datamap on the main table which has
-a <em>event_time</em> column, one datamap for one time granularity. Then Carbondata can do automatic
-roll-up for queries on the main table.</p>
+a <em>event_time</em> column, one datamap for one time granularity.</p>
 <p>For example, below statement effectively create multiple pre-aggregate tables  on main table called
 <strong>timeseries</strong></p>
 <pre><code>CREATE DATAMAP agg_year
@@ -229,26 +280,15 @@ DMPROPERTIES (
 ) 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_granularity'='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
 </code></pre>
-<p>For querying timeseries data, Carbondata has builtin support for following time related UDF
-to enable automatically roll-up to the desired aggregation level</p>
+<p>For querying timeseries data, Carbondata has builtin support for following time related UDF</p>
 <pre><code>timeseries(timeseries column name, 'aggregation level')
 </code></pre>
 <pre><code>SELECT timeseries(order_time, 'hour'), sum(quantity) FROM sales GROUP BY timeseries(order_time,
 'hour')
 </code></pre>
 <p>It is <strong>not necessary</strong> to create pre-aggregate tables for each granularity unless required for
-query. Carbondata can roll-up the data and fetch it.</p>
+query.</p>
 <p>For Example: For main table <strong>sales</strong> , if following timeseries datamaps were created for day
 level and hour level pre-aggregate</p>
 <pre><code>  CREATE DATAMAP agg_day
@@ -271,7 +311,7 @@ level and hour level pre-aggregate</p>
   SELECT order_time, country, sex, sum(quantity), max(quantity), count(user_id), sum(price),
    avg(price) FROM sales GROUP BY order_time, country, sex
 </code></pre>
-<p>Queries like below will be rolled-up and hit the timeseries datamaps</p>
+<p>Queries like below will not be rolled-up and hit the main table</p>
 <pre><code>Select timeseries(order_time, 'month'), sum(quantity) from sales group by timeseries(order_time,
   'month')
   
@@ -287,12 +327,23 @@ the future CarbonData release.</li>
 </ul>
 <h2>
 <a id="compacting-timeseries-datamp" class="anchor" href="#compacting-timeseries-datamp" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Compacting timeseries datamp</h2>
-<p>Refer to Compaction section in <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/preaggregate-datamap-guide.html" target=_blank>preaggregation datamap</a>.
+<p>Refer to Compaction section in <a href="./preaggregate-datamap-guide.html">preaggregation datamap</a>.
 Same applies to timeseries datamap.</p>
 <h2>
 <a id="data-management-on-timeseries-datamap" class="anchor" href="#data-management-on-timeseries-datamap" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Management on timeseries datamap</h2>
-<p>Refer to Data Management section in <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/preaggregate-datamap-guide.html" target=_blank>preaggregation datamap</a>.
+<p>Refer to Data Management section in <a href="./preaggregate-datamap-guide.html">preaggregation datamap</a>.
 Same applies to timeseries datamap.</p>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -308,4 +359,4 @@ Same applies to timeseries datamap.</p>
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/troubleshooting.html
----------------------------------------------------------------------
diff --git a/content/troubleshooting.html b/content/troubleshooting.html
index c668dc9..7e0cd60 100644
--- a/content/troubleshooting.html
+++ b/content/troubleshooting.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,59 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./videogallery.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__trouble nav__item" href="./troubleshooting.html">Troubleshooting</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./videogallery.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__trouble navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -348,6 +401,10 @@ For example, you can use scp to copy this file to all the nodes.</p>
 <p>Behaviour not supported.</p>
 <p><strong>Procedure</strong></p>
 <p>A single column that can be considered as dimension is mandatory for table creation.</p>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__trouble').addClass('selected'); });
+</script>
 </div>
 </div>
 </div>
@@ -363,4 +420,4 @@ For example, you can use scp to copy this file to all the nodes.</p>
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/useful-tips-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/useful-tips-on-carbondata.html b/content/useful-tips-on-carbondata.html
index 912cb48..eba1b0a 100644
--- a/content/useful-tips-on-carbondata.html
+++ b/content/useful-tips-on-carbondata.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,68 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__home nav__item" href="/">home</a>
+                            <a class="b-nav__intro nav__item" href="/intro">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="/quickstart">quickstart</a>
+                            <a class="b-nav__uses nav__item" href="/uses">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="/documentation">documentation</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#gettingStarted">getting started</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#api">APIs</a>
+                                <a class="b-nav__streams nav__item nav__sub__item" href="/documentation/streams">kafka streams</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#connect">kafka connect</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#configuration">configuration</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#design">design</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#implementation">implementation</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#operations">operations</a>
+                                <a class="nav__item nav__sub__item" href="/documentation#security">security</a>
+                            </div>
+
+                            <a class="b-nav__performance nav__item" href="/performance">performance</a>
+                            <a class="b-nav__poweredby nav__item" href="/powered-by">powered by</a>
+                            <a class="b-nav__project nav__item" href="/project">project info</a>
+                            <a class="b-nav__ecosystem nav__item" href="https://cwiki.apache.org/confluence/display/KAFKA/Ecosystem" target="_blank">ecosystem</a>
+                            <a class="b-nav__clients nav__item" href="https://cwiki.apache.org/confluence/display/KAFKA/Clients" target="_blank">clients</a>
+                            <a class="b-nav__events nav__item" href="/events">events</a>
+                            <a class="b-nav__contact nav__item" href="/contact">contact us</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__apache nav__item nav__sub__anchor b-nav__sub__anchor" href="#">apache</a>
+                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/" target="_blank">foundation</a>
+                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/licenses/" target="_blank">license</a>
+                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/foundation/sponsorship.html" target="_blank">sponsorship</a>
+                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/foundation/thanks.html" target="_blank">thanks</a>
+                                <a class="b-nav__apache nav__item nav__sub__item" href="http://www.apache.org/security/" target="_blank">security</a>
+                            </div>
+
+                            <a class="btn" href="/downloads">download</a>
+                            <div class="social-links">
+                                <a class="twitter" href="https://twitter.com/apachekafka" target="_blank">@apachekafka</a>
+                            </div>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__home navindicator__item"></div>
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__performance navindicator__item"></div>
+                        <div class="b-nav__poweredby navindicator__item"></div>
+                        <div class="b-nav__project navindicator__item"></div>
+                        <div class="b-nav__ecosystem navindicator__item"></div>
+                        <div class="b-nav__clients navindicator__item"></div>
+                        <div class="b-nav__events navindicator__item"></div>
+                        <div class="b-nav__contact navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -477,4 +539,4 @@ scenarios. After the completion of POC, some of the configurations impacting the
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/videogallery.html
----------------------------------------------------------------------
diff --git a/content/videogallery.html b/content/videogallery.html
index c282364..e494174 100644
--- a/content/videogallery.html
+++ b/content/videogallery.html
@@ -246,4 +246,4 @@
 
 
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/src/main/resources/application.conf b/src/main/resources/application.conf
index ba5b4b5..af4f234 100644
--- a/src/main/resources/application.conf
+++ b/src/main/resources/application.conf
@@ -1,16 +1,21 @@
 fileList=["configuration-parameters",
-  "data-management-on-carbondata",
    "quick-start-guide",
   "faq",
   "file-structure-of-carbondata",
-  "installation-guide",
   "streaming-guide",
   "supported-data-types-in-carbondata",
-  "troubleshooting",
-  "useful-tips-on-carbondata",
+  "performance-tuning",
   "sdk-guide",
+  "datamap-developer-guide",
+  "ddl-of-carbondata",
+  "dml-of-carbondata",
+  "documentation",
+  "language-manual",
   "s3-guide",
-  "datamap-developer-guide"
+  "segment-management-on-carbondata",
+  "release-guide",
+  "how-to-contribute-to-apache-carbondata",
+  "introduction"
   ]
 dataMapFileList=[
   "bloomfilter-datamap-guide",
@@ -21,8 +26,7 @@ dataMapFileList=[
 
 dataMapFilesUrl="https://raw.githubusercontent.com/apache/carbondata/master/docs/datamap/"
 
-fileListToRetain=["quick-start-guide",
-  "installation-guide"
+fileListToRetain=["quick-start-guide"
 ]
 headerPath="src/main/scala/html/header.html"
 footerPath="src/main/scala/html/footer.html"

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/scala/html/header.html
----------------------------------------------------------------------
diff --git a/src/main/scala/html/header.html b/src/main/scala/html/header.html
index 9e904d0..28cab9c 100644
--- a/src/main/scala/html/header.html
+++ b/src/main/scala/html/header.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/bloomfilter-datamap-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/bloomfilter-datamap-guide.html b/src/main/webapp/bloomfilter-datamap-guide.html
index e3b3eb6..54c14fe 100644
--- a/src/main/webapp/bloomfilter-datamap-guide.html
+++ b/src/main/webapp/bloomfilter-datamap-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -305,6 +356,17 @@ but if it can reduce the number of spark task,
 there is still a chance that BloomFilter datamap can enhance the performance for concurrent query.</li>
 <li>Note that BloomFilter datamap will decrease the data loading performance and may cause slightly storage expansion (for datamap index file).</li>
 </ul>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -320,4 +382,4 @@ there is still a chance that BloomFilter datamap can enhance the performance for
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>


[12/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/configuration-parameters.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/configuration-parameters.html b/src/main/webapp/configuration-parameters.html
index 4f4f1cd..ba73b0d 100644
--- a/src/main/webapp/configuration-parameters.html
+++ b/src/main/webapp/configuration-parameters.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -161,18 +212,18 @@
                                     <div>
 <h1>
 <a id="configuring-carbondata" class="anchor" href="#configuring-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configuring CarbonData</h1>
-<p>This tutorial guides you through the advanced configurations of CarbonData :</p>
+<p>This guide explains the configurations that can be used to tune CarbonData to achieve better performance.Some of the properties can be set dynamically and are explained in the section Dynamic Configuration In CarbonData Using SET-RESET.Most of the properties that control the internal settings have reasonable default values.They are listed along with the properties along with explanation.</p>
 <ul>
 <li><a href="#system-configuration">System Configuration</a></li>
-<li><a href="#performance-configuration">Performance Configuration</a></li>
-<li><a href="#miscellaneous-configuration">Miscellaneous Configuration</a></li>
-<li><a href="#spark-configuration">Spark Configuration</a></li>
+<li><a href="#data-loading-configuration">Data Loading Configuration</a></li>
+<li><a href="#compaction-configuration">Compaction Configuration</a></li>
+<li><a href="#query-configuration">Query Configuration</a></li>
+<li><a href="#data-mutation-configuration">Data Mutation Configuration</a></li>
 <li><a href="#dynamic-configuration-in-carbondata-using-set-reset">Dynamic Configuration In CarbonData Using SET-RESET</a></li>
 </ul>
 <h2>
 <a id="system-configuration" class="anchor" href="#system-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>System Configuration</h2>
 <p>This section provides the details of all the configurations required for the CarbonData System.</p>
-<p><b></b></p><p align="center">System Configuration in carbon.properties</p>
 <table>
 <thead>
 <tr>
@@ -184,411 +235,303 @@
 <tbody>
 <tr>
 <td>carbon.storelocation</td>
-<td></td>
-<td>Location where CarbonData will create the store, and write the data in its own format. If not specified then it takes spark.sql.warehouse.dir path. NOTE: Store location should be in HDFS.</td>
+<td>spark.sql.warehouse.dir property value</td>
+<td>Location where CarbonData will create the store, and write the data in its custom format. If not specified,the path defaults to spark.sql.warehouse.dir property. NOTE: Store location should be in HDFS.</td>
 </tr>
 <tr>
 <td>carbon.ddl.base.hdfs.url</td>
-<td></td>
-<td>This property is used to configure the HDFS relative path, the path configured in carbon.ddl.base.hdfs.url will be appended to the HDFS path configured in fs.defaultFS. If this path is configured, then user need not pass the complete path while dataload. For example: If absolute path of the csv file is hdfs://10.18.101.155:54310/data/cnbc/2016/xyz.csv, the path "hdfs://10.18.101.155:54310" will come from property fs.defaultFS and user can configure the /data/cnbc/ as carbon.ddl.base.hdfs.url. Now while dataload user can specify the csv path as /2016/xyz.csv.</td>
+<td>(none)</td>
+<td>To simplify and shorten the path to be specified in DDL/DML commands, this property is supported.This property is used to configure the HDFS relative path, the path configured in carbon.ddl.base.hdfs.url will be appended to the HDFS path configured in fs.defaultFS of core-site.xml. If this path is configured, then user need not pass the complete path while dataload. For example: If absolute path of the csv file is hdfs://10.18.101.155:54310/data/cnbc/2016/xyz.csv, the path "hdfs://10.18.101.155:54310" will come from property fs.defaultFS and user can configure the /data/cnbc/ as carbon.ddl.base.hdfs.url. Now while dataload user can specify the csv path as /2016/xyz.csv.</td>
 </tr>
 <tr>
 <td>carbon.badRecords.location</td>
-<td></td>
-<td>Path where the bad records are stored.</td>
-</tr>
-<tr>
-<td>carbon.data.file.version</td>
-<td>V3</td>
-<td>If this parameter value is set to 1, then CarbonData will support the data load which is in old format(0.x version). If the value is set to 2(1.x onwards version), then CarbonData will support the data load of new format only.</td>
+<td>(none)</td>
+<td>CarbonData can detect the records not conforming to defined table schema and isolate them as bad records.This property is used to specify where to store such bad records.</td>
 </tr>
 <tr>
 <td>carbon.streaming.auto.handoff.enabled</td>
 <td>true</td>
-<td>If this parameter value is set to true, auto trigger handoff function will be enabled.</td>
+<td>CarbonData supports storing of streaming data.To have high throughput for streaming, the data is written in Row format which is highly optimized for write, but performs poorly for query.When this property is true and when the streaming data size reaches <em><strong>carbon.streaming.segment.max.size</strong></em>, CabonData will automatically convert the data to columnar format and optimize it for faster querying.<strong>NOTE:</strong> It is not recommended to keep the default value which is true.</td>
 </tr>
 <tr>
 <td>carbon.streaming.segment.max.size</td>
 <td>1024000000</td>
-<td>This parameter defines the maximum size of the streaming segment. Setting this parameter to appropriate value will avoid impacting the streaming ingestion. The value is in bytes.</td>
+<td>CarbonData writes streaming data in row format which is optimized for high write throughput.This property defines the maximum size of data to be held is row format, beyond which it will be converted to columnar format in order to support high performane query, provided <em><strong>carbon.streaming.auto.handoff.enabled</strong></em> is true. <strong>NOTE:</strong> Setting higher value will impact the streaming ingestion. The value has to be configured in bytes.</td>
 </tr>
 <tr>
 <td>carbon.query.show.datamaps</td>
 <td>true</td>
-<td>If this parameter value is set to true, show tables command will list all the tables including datatmaps(eg: Preaggregate table), else datamaps will be excluded from the table list.</td>
+<td>CarbonData stores datamaps as independent tables so as to allow independent maintenance to some extent.When this property is true,which is by default, show tables command will list all the tables including datatmaps(eg: Preaggregate table), else datamaps will be excluded from the table list.<strong>NOTE:</strong>  It is generally not required for the user to do any maintenance operations on these tables and hence not required to be seen.But it is shown by default so that user or admin can get clear understanding of the system for capacity planning.</td>
 </tr>
 <tr>
 <td>carbon.segment.lock.files.preserve.hours</td>
 <td>48</td>
-<td>This property value indicates the number of hours the segment lock files will be preserved after dataload. These lock files will be deleted with the clean command after the configured number of hours.</td>
+<td>In order to support parallel data loading onto the same table, CarbonData sequences(locks) at the granularity of segments.Operations affecting the segment(like IUD, alter) are blocked from parallel operations.This property value indicates the number of hours the segment lock files will be preserved after dataload. These lock files will be deleted with the clean command after the configured number of hours.</td>
+</tr>
+<tr>
+<td>carbon.timestamp.format</td>
+<td>yyyy-MM-dd HH:mm:ss</td>
+<td>CarbonData can understand data of timestamp type and process it in special manner.It can be so that the format of Timestamp data is different from that understood by CarbonData by default.This configuration allows users to specify the format of Timestamp in their data.</td>
+</tr>
+<tr>
+<td>carbon.lock.type</td>
+<td>LOCALLOCK</td>
+<td>This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: 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: 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 HDFS supports file based locking.</td>
+</tr>
+<tr>
+<td>carbon.lock.path</td>
+<td>TABLEPATH</td>
+<td>This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3.</td>
 </tr>
 <tr>
 <td>carbon.unsafe.working.memory.in.mb</td>
 <td>512</td>
-<td>Specifies the size of executor unsafe working memory. Used for sorting data, storing column pages,etc. This value is expressed in MB.</td>
+<td>CarbonData supports storing data in off-heap memory for certain operations during data loading and query.This helps to avoid the Java GC and thereby improve the overall performance.The Minimum value recommeded is 512MB.Any value below this is reset to default value of 512MB.<strong>NOTE:</strong> The below formulas explain how to arrive at the off-heap size required.Memory Required For Data Loading:(<em>carbon.number.of.cores.while.loading</em>) * (Number of tables to load in parallel) * (<em>offheap.sort.chunk.size.inmb</em> + <em>carbon.blockletgroup.size.in.mb</em> + <em>carbon.blockletgroup.size.in.mb</em>/3.5 ). Memory required for Query:SPARK_EXECUTOR_INSTANCES * (<em>carbon.blockletgroup.size.in.mb</em> + <em>carbon.blockletgroup.size.in.mb</em> * 3.5) * spark.executor.cores</td>
 </tr>
 <tr>
-<td>carbon.unsafe.driver.working.memory.in.mb</td>
-<td>512</td>
-<td>Specifies the size of driver unsafe working memory. Used for storing block or blocklet datamap cache. If not configured then carbon.unsafe.working.memory.in.mb value is considered. This value is expressed in MB.</td>
+<td>carbon.update.sync.folder</td>
+<td>/tmp/carbondata</td>
+<td>CarbonData maintains last modification time entries in modifiedTime.htmlt to determine the schema changes and reload only when necessary.This configuration specifies the path where the file needs to be written.</td>
+</tr>
+<tr>
+<td>carbon.invisible.segments.preserve.count</td>
+<td>200</td>
+<td>CarbonData maintains each data load entry in tablestatus file. The entries from this file are not deleted for those segments that are compacted or dropped, but are made invisible.If the number of data loads are very high, the size and number of entries in tablestatus file can become too many causing unnecessary reading of all data.This configuration specifies the number of segment entries to be maintained afte they are compacted or dropped.Beyond this, the entries are moved to a separate history tablestatus file.<strong>NOTE:</strong> The entries in tablestatus file help to identify the operations performed on CarbonData table and is also used for checkpointing during various data manupulation operations.This is similar to AUDIT file maintaining all the operations and its status.Hence the entries are never deleted but moved to a separate history file.</td>
+</tr>
+<tr>
+<td>carbon.lock.retries</td>
+<td>3</td>
+<td>CarbonData ensures consistency of operations by blocking certain operations from running in parallel.In order to block the operations from running in parallel, lock is obtained on the table.This configuration specifies the maximum number of retries to obtain the lock for any operations other than load.<strong>NOTE:</strong> Data manupulation operations like Compaction,UPDATE,DELETE  or LOADING,UPDATE,DELETE are not allowed to run in parallel.How ever data loading can happen in parallel to compaction.</td>
+</tr>
+<tr>
+<td>carbon.lock.retry.timeout.sec</td>
+<td>5</td>
+<td>Specifies the interval between the retries to obtain the lock for any operation other than load.<strong>NOTE:</strong> Refer to <em><strong>carbon.lock.retries</strong></em> for understanding why CarbonData uses locks for operations.</td>
 </tr>
 </tbody>
 </table>
 <h2>
-<a id="performance-configuration" class="anchor" href="#performance-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Performance Configuration</h2>
-<p>This section provides the details of all the configurations required for CarbonData Performance Optimization.</p>
-<p><b></b></p><p align="center">Performance Configuration in carbon.properties</p>
-<ul>
-<li><strong>Data Loading Configuration</strong></li>
-</ul>
+<a id="data-loading-configuration" class="anchor" href="#data-loading-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Loading Configuration</h2>
 <table>
 <thead>
 <tr>
 <th>Parameter</th>
 <th>Default Value</th>
 <th>Description</th>
-<th>Range</th>
 </tr>
 </thead>
 <tbody>
 <tr>
 <td>carbon.number.of.cores.while.loading</td>
 <td>2</td>
-<td>Number of cores to be used while loading data.</td>
-<td></td>
+<td>Number of cores to be used while loading data.This also determines the number of threads to be used to read the input files (csv) in parallel.<strong>NOTE:</strong> This configured value is used in every data loading step to parallelize the operations. Configuring a higher value can lead to increased early thread pre-emption by OS and there by reduce the overall performance.</td>
 </tr>
 <tr>
 <td>carbon.sort.size</td>
 <td>100000</td>
-<td>Record count to sort and write intermediate files to temp.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.max.driver.lru.cache.size</td>
-<td>-1</td>
-<td>Max LRU cache size upto which data will be loaded at the driver side. This value is expressed in MB. Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.max.executor.lru.cache.size</td>
-<td>-1</td>
-<td>Max LRU cache size upto which data will be loaded at the executor side. This value is expressed in MB. Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted. If this parameter is not configured, then the carbon.max.driver.lru.cache.size value will be considered.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.merge.sort.prefetch</td>
-<td>true</td>
-<td>Enable prefetch of data during merge sort while reading data from sort temp files in data loading.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.insert.persist.enable</td>
-<td>false</td>
-<td>Enabling this parameter considers persistent data. If we are executing insert into query from source table using select statement &amp; loading the same source table concurrently, when select happens on source table during the data load, it gets new record for which dictionary is not generated, so there will be inconsistency. To avoid this condition we can persist the dataframe into MEMORY_AND_DISK(default value) and perform insert into operation. By default this value will be false because no need to persist the dataframe in all cases. If user wants to run load and insert queries on source table concurrently then user can enable this parameter.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.insert.storage.level</td>
-<td>MEMORY_AND_DISK</td>
-<td>Which storage level to persist dataframe when 'carbon.insert.persist.enable'=true, if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. <a href="http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence" rel="nofollow">See detail</a>.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.update.persist.enable</td>
-<td>true</td>
-<td>Enabling this parameter considers persistent data. Enabling this will reduce the execution time of UPDATE operation.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.update.storage.level</td>
-<td>MEMORY_AND_DISK</td>
-<td>Which storage level to persist dataframe when 'carbon.update.persist.enable'=true, if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. <a href="http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence" rel="nofollow">See detail</a>.</td>
-<td></td>
+<td>Number of records to hold in memory to sort and write intermediate temp files.<strong>NOTE:</strong> Memory required for data loading increases with increase in configured value as each thread would cache configured number of records.</td>
 </tr>
 <tr>
 <td>carbon.global.sort.rdd.storage.level</td>
 <td>MEMORY_ONLY</td>
-<td>Which storage level to persist rdd when loading data with 'sort_scope'='global_sort', if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. <a href="http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence" rel="nofollow">See detail</a>.</td>
-<td></td>
+<td>Storage level to persist dataset of RDD/dataframe when loading data with 'sort_scope'='global_sort', if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. <a href="http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence" rel="nofollow">See detail</a>.</td>
 </tr>
 <tr>
 <td>carbon.load.global.sort.partitions</td>
 <td>0</td>
-<td>The Number of partitions to use when shuffling data for sort. If user don't configurate or configurate it less than 1, it uses the number of map tasks as reduce tasks. In general, we recommend 2-3 tasks per CPU core in your cluster.</td>
-<td></td>
+<td>The Number of partitions to use when shuffling data for sort. Default value 0 means to use same number of map tasks as reduce tasks.<strong>NOTE:</strong> In general, it is recommended to have 2-3 tasks per CPU core in your cluster.</td>
 </tr>
 <tr>
 <td>carbon.options.bad.records.logger.enable</td>
 <td>false</td>
-<td>Whether to create logs with details about bad records.</td>
-<td></td>
+<td>CarbonData can identify the records that are not conformant to schema and isolate them as bad records.Enabling this configuration will make CarbonData to log such bad records.<strong>NOTE:</strong> If the input data contains many bad records, logging them will slow down the over all data loading throughput.The data load operation status would depend on the configuration in <em><strong>carbon.bad.records.action</strong></em>.</td>
 </tr>
 <tr>
 <td>carbon.bad.records.action</td>
-<td>FORCE</td>
-<td>This property can have four types of actions for bad records FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found.</td>
-<td></td>
+<td>FAIL</td>
+<td>CarbonData in addition to identifying the bad records, can take certain actions on such data.This configuration can have four types of actions for bad records namely FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found.</td>
 </tr>
 <tr>
 <td>carbon.options.is.empty.data.bad.record</td>
 <td>false</td>
-<td>If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa.</td>
-<td></td>
+<td>Based on the business scenarios, empty("" or '' or ,,) data can be valid or invalid. This configuration controls how empty data should be treated by CarbonData. If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa.</td>
 </tr>
 <tr>
 <td>carbon.options.bad.record.path</td>
-<td></td>
-<td>Specifies the HDFS path where bad records are stored. By default the value is Null. This path must to be configured by the user if bad record logger is enabled or bad record action redirect.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.enable.vector.reader</td>
-<td>true</td>
-<td>This parameter increases the performance of select queries as it fetch columnar batch of size 4*1024 rows instead of fetching data row by row.</td>
-<td></td>
+<td>(none)</td>
+<td>Specifies the HDFS path where bad records are to be stored. By default the value is Null. This path must to be configured by the user if <em><strong>carbon.options.bad.records.logger.enable</strong></em> is <strong>true</strong> or <em><strong>carbon.bad.records.action</strong></em> is <strong>REDIRECT</strong>.</td>
 </tr>
 <tr>
 <td>carbon.blockletgroup.size.in.mb</td>
-<td>64 MB</td>
-<td>The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of the blocklet group. Higher value results in better sequential IO access.The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB).</td>
-<td></td>
+<td>64</td>
+<td>Please refer to <a href="./file-structure-of-carbondata.html">file-structure-of-carbondata</a> to understand the storage format of CarbonData.The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of each blocklet group. Higher value results in better sequential IO access.The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB).<strong>NOTE:</strong> Configuring a higher value might lead to poor performance as an entire blocklet group will have to read into memory before processing.For filter queries with limit, it is <strong>not advisable</strong> to have a bigger blocklet size.For Aggregation queries which need to return more number of rows,bigger blocklet size is advisable.</td>
 </tr>
 <tr>
-<td>carbon.task.distribution</td>
-<td>block</td>
-<td>
-<strong>block</strong>: Setting this value will launch one task per block. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. <strong>custom</strong>: Setting this value will group the blocks and distribute it uniformly to the available resources in the cluster. This enhances the query performance but not suggested in case of concurrent queries and queries having big shuffling scenarios. <strong>blocklet</strong>: Setting this value will launch one task per blocklet. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. <strong>merge_small_files</strong>: Setting this value will merge all the small partitions to a size of (128 MB is the default value of "spark.sql.files.maxPartitionBytes",it is configurable) during querying. The small partitions are combined to a map task to reduce the number of read task. This enhances the performance.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.load.sortmemory.spill.percentage</td>
-<td>0</td>
-<td>If we use unsafe memory during data loading, this configuration will be used to control the behavior of spilling inmemory pages to disk. Internally in Carbondata, during sorting carbondata will sort data in pages and add them in unsafe memory. If the memory is insufficient, carbondata will spill the pages to disk and generate sort temp file. This configuration controls how many pages in memory will be spilled to disk based size. The size can be calculated by multiplying this configuration value with 'carbon.sort.storage.inmemory.size.inmb'. For example, default value 0 means that no pages in unsafe memory will be spilled and all the newly sorted data will be spilled to disk; Value 50 means that if the unsafe memory is insufficient, about half of pages in the unsafe memory will be spilled to disk while value 100 means that almost all pages in unsafe memory will be spilled. <strong>Note</strong>: This configuration only works for 'LOCAL_SORT' and 'BATCH_SORT' and the actual spilli
 ng behavior may slightly be different in each data loading.</td>
-<td>Integer values between 0 and 100</td>
-</tr>
-</tbody>
-</table>
-<ul>
-<li><strong>Compaction Configuration</strong></li>
-</ul>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Default Value</th>
-<th>Description</th>
-<th>Range</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>carbon.number.of.cores.while.compacting</td>
-<td>2</td>
-<td>Number of cores which are used to write data during compaction.</td>
-<td></td>
+<td>carbon.sort.file.write.buffer.size</td>
+<td>16384</td>
+<td>CarbonData sorts and writes data to intermediate files to limit the memory usage.This configuration determines the buffer size to be used for reading and writing such files. <strong>NOTE:</strong> This configuration is useful to tune IO and derive optimal performance.Based on the OS and underlying harddisk type, these values can significantly affect the overall performance.It is ideal to tune the buffersize equivalent to the IO buffer size of the OS.Recommended range is between 10240 to 10485760 bytes.</td>
 </tr>
 <tr>
-<td>carbon.compaction.level.threshold</td>
-<td>4, 3</td>
-<td>This property is for minor compaction which decides how many segments to be merged. Example: If it is set as 2, 3 then minor compaction will be triggered for every 2 segments. 3 is the number of level 1 compacted segment which is further compacted to new segment.</td>
-<td>Valid values are from 0-100.</td>
+<td>carbon.sort.intermediate.files.limit</td>
+<td>20</td>
+<td>CarbonData sorts and writes data to intermediate files to limit the memory usage.Before writing the target carbondat file, the data in these intermediate files needs to be sorted again so as to ensure the entire data in the data load is sorted.This configuration determines the minimum number of intermediate files after which merged sort is applied on them sort the data.<strong>NOTE:</strong> Intermediate merging happens on a separate thread in the background.Number of threads used is determined by <em><strong>carbon.merge.sort.reader.thread</strong></em>.Configuring a low value will cause more time to be spent in merging these intermediate merged files which can cause more IO.Configuring a high value would cause not to use the idle threads to do intermediate sort merges.Range of recommended values are between 2 and 50</td>
 </tr>
 <tr>
-<td>carbon.major.compaction.size</td>
-<td>1024</td>
-<td>Major compaction size can be configured using this parameter. Sum of the segments which is below this threshold will be merged. This value is expressed in MB.</td>
-<td></td>
+<td>carbon.csv.read.buffersize.byte</td>
+<td>1048576</td>
+<td>CarbonData uses Hadoop InputFormat to read the csv files.This configuration value is used to pass buffer size as input for the Hadoop MR job when reading the csv files.This value is configured in bytes.<strong>NOTE:</strong> Refer to <em><strong>org.apache.hadoop.mapreduce.InputFormat</strong></em> documentation for additional information.</td>
 </tr>
 <tr>
-<td>carbon.horizontal.compaction.enable</td>
-<td>true</td>
-<td>This property is used to turn ON/OFF horizontal compaction. After every DELETE and UPDATE statement, horizontal compaction may occur in case the delta (DELETE/ UPDATE) files becomes more than specified threshold.</td>
-<td></td>
+<td>carbon.merge.sort.reader.thread</td>
+<td>3</td>
+<td>CarbonData sorts and writes data to intermediate files to limit the memory usage.When the intermediate files reaches <em><strong>carbon.sort.intermediate.files.limit</strong></em> the files will be merged,the number of threads specified in this configuration will be used to read the intermediate files for performing merge sort.<strong>NOTE:</strong> Refer to <em><strong>carbon.sort.intermediate.files.limit</strong></em> for operation description.Configuring less  number of threads can cause merging to slow down over loading process where as configuring more number of threads can cause thread contention with threads in other data loading steps.Hence configure a fraction of <em><strong>carbon.number.of.cores.while.loading</strong></em>.</td>
 </tr>
 <tr>
-<td>carbon.horizontal.UPDATE.compaction.threshold</td>
-<td>1</td>
-<td>This property specifies the threshold limit on number of UPDATE delta files within a segment. In case the number of delta files goes beyond the threshold, the UPDATE delta files within the segment becomes eligible for horizontal compaction and compacted into single UPDATE delta file.</td>
-<td>Values between 1 to 10000.</td>
+<td>carbon.concurrent.lock.retries</td>
+<td>100</td>
+<td>CarbonData supports concurrent data loading onto same table.To ensure the loading status is correctly updated into the system,locks are used to sequence the status updation step.This configuration specifies the maximum number of retries to obtain the lock for updating the load status.<strong>NOTE:</strong> This value is high as more number of concurrent loading happens,more the chances of not able to obtain the lock when tried.Adjust this value according to the number of concurrent loading to be supported by the system.</td>
 </tr>
 <tr>
-<td>carbon.horizontal.DELETE.compaction.threshold</td>
+<td>carbon.concurrent.lock.retry.timeout.sec</td>
 <td>1</td>
-<td>This property specifies the threshold limit on number of DELETE delta files within a block of a segment. In case the number of delta files goes beyond the threshold, the DELETE delta files for the particular block of the segment becomes eligible for horizontal compaction and compacted into single DELETE delta file.</td>
-<td>Values between 1 to 10000.</td>
+<td>Specifies the interval between the retries to obtain the lock for concurrent operations.<strong>NOTE:</strong> Refer to <em><strong>carbon.concurrent.lock.retries</strong></em> for understanding why CarbonData uses locks during data loading operations.</td>
 </tr>
 <tr>
-<td>carbon.update.segment.parallelism</td>
-<td>1</td>
-<td>This property specifies the parallelism for each segment during update. If there are segments that contain too many records to update and the spark job encounter data-spill related errors, it is better to increase this property value. It is recommended to set this value to a multiple of the number of executors for balance.</td>
-<td>Values between 1 to 1000.</td>
+<td>carbon.skip.empty.line</td>
+<td>false</td>
+<td>The csv files givent to CarbonData for loading can contain empty lines.Based on the business scenario, this empty line might have to be ignored or needs to be treated as NULL value for all columns.In order to define this business behavior, this configuration is provided.<strong>NOTE:</strong> In order to consider NULL values for non string columns and continue with data load, <em><strong>carbon.bad.records.action</strong></em> need to be set to <strong>FORCE</strong>;else data load will be failed as bad records encountered.</td>
 </tr>
 <tr>
-<td>carbon.merge.index.in.segment</td>
+<td>carbon.enable.calculate.size</td>
 <td>true</td>
-<td>This property is used to merge all carbon index files (.carbonindex) inside a segment to a single carbon index merge file (.carbonindexmerge).</td>
-<td>Values true or false</td>
+<td>
+<strong>For Load Operation</strong>: Setting this property calculates the size of the carbon data file (.carbondata) and carbon index file (.carbonindex) for every load and updates the table status file. <strong>For Describe Formatted</strong>: Setting this property calculates the total size of the carbon data files and carbon index files for the respective table and displays in describe formatted command.<strong>NOTE:</strong> This is useful to determine the overall size of the carbondata table and also get an idea of how the table is growing in order to take up other backup strategy decisions.</td>
 </tr>
-</tbody>
-</table>
-<ul>
-<li><strong>Query Configuration</strong></li>
-</ul>
-<table>
-<thead>
 <tr>
-<th>Parameter</th>
-<th>Default Value</th>
-<th>Description</th>
-<th>Range</th>
+<td>carbon.cutOffTimestamp</td>
+<td>(none)</td>
+<td>CarbonData has capability to generate the Dictionary values for the timestamp columns from the data itself without the need to store the computed dictionary values. This configuration sets the start date for calculating the timestamp. Java counts the number of milliseconds from start of "1970-01-01 00:00:00". This property is used to customize the start of position. For example "2000-01-01 00:00:00". <strong>NOTE:</strong> The date must be in the form <em><strong>carbon.timestamp.format</strong></em>. CarbonData supports storing data for upto 68 years.For example, if the cut-off time is 1970-01-01 05:30:00, then data upto 2038-01-01 05:30:00 will be supported by CarbonData.</td>
 </tr>
-</thead>
-<tbody>
 <tr>
-<td>carbon.number.of.cores</td>
-<td>4</td>
-<td>Number of cores to be used while querying.</td>
-<td></td>
+<td>carbon.timegranularity</td>
+<td>SECOND</td>
+<td>The configuration is used to specify the data granularity level such as DAY, HOUR, MINUTE, or SECOND.This helps to store more than 68 years of data into CarbonData.</td>
 </tr>
 <tr>
-<td>carbon.enable.quick.filter</td>
+<td>carbon.use.local.dir</td>
 <td>false</td>
-<td>Improves the performance of filter query.</td>
-<td></td>
+<td>CarbonData during data loading, writes files to local temp directories before copying the files to HDFS.This configuration is used to specify whether CarbonData can write locally to tmp directory of the container or to the YARN application directory.</td>
 </tr>
-</tbody>
-</table>
-<h2>
-<a id="miscellaneous-configuration" class="anchor" href="#miscellaneous-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Miscellaneous Configuration</h2>
-<p><b></b></p><p align="center">Extra Configuration in carbon.properties</p>
-<ul>
-<li><strong>Time format for CarbonData</strong></li>
-</ul>
-<table>
-<thead>
 <tr>
-<th>Parameter</th>
-<th>Default Format</th>
-<th>Description</th>
+<td>carbon.use.multiple.temp.dir</td>
+<td>false</td>
+<td>When multiple disks are present in the system, YARN is generally configured with multiple disks to be used as temp directories for managing the containers.This configuration specifies whether to use multiple YARN local directories during data loading for disk IO load balancing.Enable <em><strong>carbon.use.local.dir</strong></em> for this configuration to take effect.<strong>NOTE:</strong> Data Loading is an IO intensive operation whose performance can be limited by the disk IO threshold, particularly during multi table concurrent data load.Configuring this parameter, balances the disk IO across multiple disks there by improving the over all load performance.</td>
 </tr>
-</thead>
-<tbody>
 <tr>
-<td>carbon.timestamp.format</td>
-<td>yyyy-MM-dd HH:mm:ss</td>
-<td>Timestamp format of input data used for timestamp data type.</td>
+<td>carbon.sort.temp.compressor</td>
+<td>(none)</td>
+<td>CarbonData writes every <em><strong>carbon.sort.size</strong></em> number of records to intermediate temp files during data loading to ensure memory footprint is within limits.These temporary files cab be compressed and written in order to save the storage space.This configuration specifies the name of compressor to be used to compress the intermediate sort temp files during sort procedure in data loading.The valid values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files.<strong>NOTE:</strong> Compressor will be useful if you encounter disk bottleneck.Since the data needs to be compressed and decompressed,it involves additional CPU cycles,but is compensated by the high IO throughput due to less data to be written or read from the disks.</td>
 </tr>
-</tbody>
-</table>
-<ul>
-<li><strong>Dataload Configuration</strong></li>
-</ul>
-<table>
-<thead>
 <tr>
-<th>Parameter</th>
-<th>Default Value</th>
-<th>Description</th>
+<td>carbon.load.skewedDataOptimization.enabled</td>
+<td>false</td>
+<td>During data loading,CarbonData would divide the number of blocks equally so as to ensure all executors process same number of blocks.This mechanism satisfies most of the scenarios and ensures maximum parallel processing for optimal data loading performance.In some business scenarios, there might be scenarios where the size of blocks vary significantly and hence some executors would have to do more work if they get blocks containing more data. This configuration enables size based block allocation strategy for data loading.When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data.<strong>NOTE:</strong> This configuration is useful if the size of your input data files varies widely, say 1MB~1GB.For this configuration to work effectively,knowing the data pattern and size is important and necessary.</td>
 </tr>
-</thead>
-<tbody>
 <tr>
-<td>carbon.sort.file.write.buffer.size</td>
-<td>16384</td>
-<td>File write buffer size used during sorting. Minimum allowed buffer size is 10240 byte and Maximum allowed buffer size is 10485760 byte.</td>
+<td>carbon.load.min.size.enabled</td>
+<td>false</td>
+<td>During Data Loading, CarbonData would divide the number of files among the available executors to parallelize the loading operation.When the input data files are very small, this action causes to generate many small carbondata files.This configuration determines whether to enable node minumun input data size allocation strategy for data loading.It will make sure that the node load the minimum amount of data there by reducing number of carbondata files.<strong>NOTE:</strong> This configuration is useful if the size of the input data files are very small, like 1MB~256MB.Refer to <em><strong>load_min_size_inmb</strong></em> to configure the minimum size to be considered for splitting files among executors.</td>
 </tr>
 <tr>
-<td>carbon.lock.type</td>
-<td>LOCALLOCK</td>
-<td>This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: 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: 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 HDFS supports file based locking.</td>
+<td>enable.data.loading.statistics</td>
+<td>false</td>
+<td>CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues.This configuration when made <em><strong>true</strong></em> would log additional data loading statistics information to more accurately locate the issues being debugged.<strong>NOTE:</strong> Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time.It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately.Also extensive logging is an increased IO operation and hence over all data loading performance might get reduced.Therefore it is recommened to enable this configuration only for the duration of debugging.</td>
 </tr>
 <tr>
-<td>carbon.lock.path</td>
-<td>TABLEPATH</td>
-<td>Locks on the files are used to prevent concurrent operation from modifying the same files. This</td>
+<td>carbon.dictionary.chunk.size</td>
+<td>10000</td>
+<td>CarbonData generates dictionary keys and writes them to separate dictionary file during data loading.To optimize the IO, this configuration determines the number of dictionary keys to be persisted to dictionary file at a time.<strong>NOTE:</strong> Writing to file also serves as a commit point to the dictionary generated.Increasing more values in memory causes more data loss during system or application failure.It is advised to alter this configuration judiciously.</td>
 </tr>
 <tr>
-<td>configuration specifies the path where lock files have to be created. Recommended to configure</td>
-<td></td>
-<td></td>
+<td>dictionary.worker.threads</td>
+<td>1</td>
+<td>CarbonData supports Optimized data loading by relying on a dictionary server.Dictionary server helps  to maintain dictionary values independent of the data loading and there by avoids reading the same input data multiples times.This configuration determines the number of concurrent dictionary generation or request that needs to be served by the dictionary server.<strong>NOTE:</strong> This configuration takes effect when <em><strong>carbon.options.single.pass</strong></em> is configured as true.Please refer to <em>carbon.options.single.pass</em>to understand how dictionary server optimizes data loading.</td>
 </tr>
 <tr>
-<td>HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3.</td>
-<td></td>
-<td></td>
+<td>enable.unsafe.sort</td>
+<td>true</td>
+<td>CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations.This configuration enables to use unsafe functions in CarbonData.<strong>NOTE:</strong> For operations like data loading, which generates more short lived Java objects, Java GC can be a bottle neck.Using unsafe can overcome the GC overhead and improve the overall performance.</td>
 </tr>
 <tr>
-<td>
-<strong>Note:</strong> If this property is not set to HDFS location for S3 store, then there is a possibility</td>
-<td></td>
-<td></td>
+<td>enable.offheap.sort</td>
+<td>true</td>
+<td>CarbonData supports storing data in off-heap memory for certain operations during data loading and query.This helps to avoid the Java GC and thereby improve the overall performance.This configuration enables using off-heap memory for sorting of data during data loading.<strong>NOTE:</strong>  <em><strong>enable.unsafe.sort</strong></em> configuration needs to be configured to true for using off-heap</td>
 </tr>
 <tr>
-<td>of data corruption because multiple data manipulation calls might try to update the status file</td>
-<td></td>
-<td></td>
+<td>enable.inmemory.merge.sort</td>
+<td>false</td>
+<td>CarbonData sorts and writes data to intermediate files to limit the memory usage.These intermediate files needs to be sorted again using merge sort before writing to the final carbondata file.Performing merge sort in memory would increase the sorting performance at the cost of increased memory footprint. This Configuration specifies to do in-memory merge sort or to do file based merge sort.</td>
 </tr>
 <tr>
-<td>and as lock is not acquired before updation data might get overwritten.</td>
-<td></td>
-<td></td>
+<td>carbon.load.sort.scope</td>
+<td>LOCAL_SORT</td>
+<td>CarbonData can support various sorting options to match the balance between load and query performance.LOCAL_SORT:All the data given to an executor in the single load is fully sorted and written to carondata files.Data loading performance is reduced a little as the entire data needs to be sorted in the executor.BATCH_SORT:Sorts the data in batches of configured size and writes to carbondata files.Data loading performance increases as the entire data need not be sorted.But query performance will get reduced due to false positives in block pruning and also due to more number of carbondata files written.Due to more number of carbondata files, if identified blocks &gt; cluster parallelism, query performance and concurrency will get reduced.GLOBAL SORT:Entire data in the data load is fully sorted and written to carbondata files.Data loading perfromance would get reduced as the entire data needs to be sorted.But the query performance increases significantly due to very less false posi
 tives and concurrency is also improved.<strong>NOTE:</strong> when BATCH_SORTis configured, it is recommended to keep <em><strong>carbon.load.batch.sort.size.inmb</strong></em> &gt; <em><strong>carbon.blockletgroup.size.in.mb</strong></em>
+</td>
 </tr>
 <tr>
-<td>carbon.sort.intermediate.files.limit</td>
-<td>20</td>
-<td>Minimum number of intermediate files after which merged sort can be started (minValue = 2, maxValue=50).</td>
+<td>carbon.load.batch.sort.size.inmb</td>
+<td>0</td>
+<td>When  <em><strong>carbon.load.sort.scope</strong></em> is configured as <em><strong>BATCH_SORT</strong></em>,This configuration needs to be added to specify the batch size for sorting and writing to carbondata files.<strong>NOTE:</strong> It is recommended to keep the value around 45% of <em><strong>carbon.sort.storage.inmemory.size.inmb</strong></em> to avoid spill to disk.Also it is recommended to keep the value higher than <em><strong>carbon.blockletgroup.size.in.mb</strong></em>. Refer to <em>carbon.load.sort.scope</em> for more information on sort options and the advantages/disadvantges of each option.</td>
 </tr>
 <tr>
-<td>carbon.block.meta.size.reserved.percentage</td>
-<td>10</td>
-<td>Space reserved in percentage for writing block meta data in CarbonData file.</td>
+<td>carbon.dictionary.server.port</td>
+<td>2030</td>
+<td>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.Single pass loading can be enabled using the option <em><strong>carbon.options.single.pass</strong></em>.When this option is specified, a dictionary server will be internally started to handle the dictionary generation and query requests.This configuration specifies the port on which the server need to listen for incoming requests.Port value ranges between 0-65535</td>
 </tr>
 <tr>
-<td>carbon.csv.read.buffersize.byte</td>
-<td>1048576</td>
-<td>csv reading buffer size.</td>
+<td>carbon.merge.sort.prefetch</td>
+<td>true</td>
+<td>CarbonData writes every <em><strong>carbon.sort.size</strong></em> number of records to intermediate temp files during data loading to ensure memory footprint is within limits.These intermediate temp files will have to be sorted using merge sort before writing into CarbonData format.This configuration enables pre fetching of data from these temp files in order to optimize IO and speed up data loading process.</td>
 </tr>
 <tr>
-<td>carbon.merge.sort.reader.thread</td>
-<td>3</td>
-<td>Maximum no of threads used for reading intermediate files for final merging.</td>
+<td>carbon.loading.prefetch</td>
+<td>false</td>
+<td>CarbonData uses univocity parser to read csv files.This configuration is used to inform the parser whether it can prefetch the data from csv files to speed up the reading.<strong>NOTE:</strong> Enabling prefetch improves the data loading performance, but needs higher memory to keep more records which are read ahead from disk.</td>
 </tr>
 <tr>
-<td>carbon.concurrent.lock.retries</td>
-<td>100</td>
-<td>Specifies the maximum number of retries to obtain the lock for concurrent operations. This is used for concurrent loading.</td>
+<td>carbon.prefetch.buffersize</td>
+<td>1000</td>
+<td>When the configuration <em><strong>carbon.merge.sort.prefetch</strong></em> is configured to true, we need to set the number of records that can be prefetched.This configuration is used specify the number of records to be prefetched.**NOTE: **Configuring more number of records to be prefetched increases memory footprint as more records will have to be kept in memory.</td>
 </tr>
 <tr>
-<td>carbon.concurrent.lock.retry.timeout.sec</td>
-<td>1</td>
-<td>Specifies the interval between the retries to obtain the lock for concurrent operations.</td>
+<td>load_min_size_inmb</td>
+<td>256</td>
+<td>This configuration is used along with <em><strong>carbon.load.min.size.enabled</strong></em>.This determines the minimum size of input files to be considered for distribution among executors while data loading.<strong>NOTE:</strong> Refer to <em><strong>carbon.load.min.size.enabled</strong></em> for understanding when this configuration needs to be used and its advantages and disadvantages.</td>
 </tr>
 <tr>
-<td>carbon.lock.retries</td>
-<td>3</td>
-<td>Specifies the maximum number of retries to obtain the lock for any operations other than load.</td>
+<td>carbon.load.sortmemory.spill.percentage</td>
+<td>0</td>
+<td>During data loading, some data pages are kept in memory upto memory configured in <em><strong>carbon.sort.storage.inmemory.size.inmb</strong></em> beyond which they are spilled to disk as intermediate temporary sort files.This configuration determines after what percentage data needs to be spilled to disk.<strong>NOTE:</strong> Without this configuration, when the data pages occupy upto configured memory, new data pages would be dumped to disk and old pages are still maintained in disk.</td>
 </tr>
 <tr>
-<td>carbon.lock.retry.timeout.sec</td>
-<td>5</td>
-<td>Specifies the interval between the retries to obtain the lock for any operation other than load.</td>
+<td>carbon.load.directWriteHdfs.enabled</td>
+<td>false</td>
+<td>During data load all the carbondata files are written to local disk and finally copied to the target location in HDFS.Enabling this parameter will make carrbondata files to be written directly onto target HDFS location bypassing the local disk.<strong>NOTE:</strong> Writing directly to HDFS saves local disk IO(once for writing the files and again for copying to HDFS) there by improving the performance.But the drawback is when data loading fails or the application crashes, unwanted carbondata files will remain in the target HDFS location until it is cleared during next data load or by running <em>CLEAN FILES</em> DDL command</td>
 </tr>
 <tr>
-<td>carbon.skip.empty.line</td>
-<td>false</td>
-<td>Setting this property ignores the empty lines in the CSV file during the data load</td>
+<td>carbon.options.serialization.null.format</td>
+<td>\N</td>
+<td>Based on the business scenarios, some columns might need to be loaded with null values.As null value cannot be written in csv files, some special characters might be adopted to specify null values.This configuration can be used to specify the null values format in the data being loaded.</td>
 </tr>
 <tr>
-<td>carbon.enable.calculate.size</td>
-<td>true</td>
-<td>
-<strong>For Load Operation</strong>: Setting this property calculates the size of the carbon data file (.carbondata) and carbon index file (.carbonindex) for every load and updates the table status file. <strong>For Describe Formatted</strong>: Setting this property calculates the total size of the carbon data files and carbon index files for the respective table and displays in describe formatted command.</td>
+<td>carbon.sort.storage.inmemory.size.inmb</td>
+<td>512</td>
+<td>CarbonData writes every <em><strong>carbon.sort.size</strong></em> number of records to intermediate temp files during data loading to ensure memory footprint is within limits.When <em><strong>enable.unsafe.sort</strong></em> configuration is enabled, instead of using <em><strong>carbon.sort.size</strong></em> which is based on rows count, size occupied in memory is used to determine when to flush data pages to intermediate temp files.This configuration determines the memory to be used for storing data pages in memory.<strong>NOTE:</strong> Configuring a higher values ensures more data is maintained in memory and hence increases data loading performance due to reduced or no IO.Based on the memory availability in the nodes of the cluster, configure the values accordingly.</td>
 </tr>
 </tbody>
 </table>
-<ul>
-<li><strong>Compaction Configuration</strong></li>
-</ul>
+<h2>
+<a id="compaction-configuration" class="anchor" href="#compaction-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Compaction Configuration</h2>
 <table>
 <thead>
 <tr>
@@ -599,30 +542,80 @@
 </thead>
 <tbody>
 <tr>
+<td>carbon.number.of.cores.while.compacting</td>
+<td>2</td>
+<td>Number of cores to be used while compacting data.This also determines the number of threads to be used to read carbondata files in parallel.</td>
+</tr>
+<tr>
+<td>carbon.compaction.level.threshold</td>
+<td>4, 3</td>
+<td>Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance.This configuration is for minor compaction which decides how many segments to be merged. Configuration is of the form (x,y). Compaction will be triggered for every x segments and form a single level 1 compacted segment.When the number of compacted level 1 segments reach y, compaction will be triggered again to merge them to form a single level 2 segment. For example: If it is set as 2, 3 then minor compaction will be triggered for every 2 segments. 3 is the number of level 1 compacted segments which is further compacted to new segment.<strong>NOTE:</strong> When <em><strong>carbon.enable.auto.load.merge</strong></em> is <strong>true</strong>, Configuring higher values cause overall data loading time to increase as compaction will be triggered after data loading is complete but status is not returned till compaction is
  complete. But compacting more number of segments can increase query performance.Hence optimal values needs to be configured based on the business scenario.Valid values are bwteen 0 to 100.</td>
+</tr>
+<tr>
+<td>carbon.major.compaction.size</td>
+<td>1024</td>
+<td>To improve query performance and All the segments can be merged and compacted to a single segment upto configured size.This Major compaction size can be configured using this parameter. Sum of the segments which is below this threshold will be merged. This value is expressed in MB.</td>
+</tr>
+<tr>
+<td>carbon.horizontal.compaction.enable</td>
+<td>true</td>
+<td>CarbonData supports DELETE/UPDATE functionality by creating delta data files for existing carbondata files.These delta files would grow as more number of DELETE/UPDATE operations are performed.Compaction of these delta files are termed as horizontal compaction.This configuration is used to turn ON/OFF horizontal compaction. After every DELETE and UPDATE statement, horizontal compaction may occur in case the delta (DELETE/ UPDATE) files becomes more than specified threshold.**NOTE: **Having many delta files will reduce the query performance as scan has to happen on all these files before the final state of data can be decided.Hence it is advisable to keep horizontal compaction enabled and configure reasonable values to <em><strong>carbon.horizontal.UPDATE.compaction.threshold</strong></em> and <em><strong>carbon.horizontal.DELETE.compaction.threshold</strong></em>
+</td>
+</tr>
+<tr>
+<td>carbon.horizontal.update.compaction.threshold</td>
+<td>1</td>
+<td>This configuration specifies the threshold limit on number of UPDATE delta files within a segment. In case the number of delta files goes beyond the threshold, the UPDATE delta files within the segment becomes eligible for horizontal compaction and are compacted into single UPDATE delta file.Values range between 1 to 10000.</td>
+</tr>
+<tr>
+<td>carbon.horizontal.delete.compaction.threshold</td>
+<td>1</td>
+<td>This configuration specifies the threshold limit on number of DELETE delta files within a block of a segment. In case the number of delta files goes beyond the threshold, the DELETE delta files for the particular block of the segment becomes eligible for horizontal compaction and are compacted into single DELETE delta file.Values range between 1 to 10000.</td>
+</tr>
+<tr>
+<td>carbon.update.segment.parallelism</td>
+<td>1</td>
+<td>CarbonData processes the UPDATE operations by grouping records belonging to a segment into a single executor task.When the amount of data to be updated is more, this behavior causes problems like restarting of executor due to low memory and data-spill related errors.This property specifies the parallelism for each segment during update.<strong>NOTE:</strong> It is recommended to set this value to a multiple of the number of executors for balance.Values range between 1 to 1000.</td>
+</tr>
+<tr>
 <td>carbon.numberof.preserve.segments</td>
 <td>0</td>
-<td>If the user wants to preserve some number of segments from being compacted then he can set this property. Example: carbon.numberof.preserve.segments = 2 then 2 latest segments will always be excluded from the compaction. No segments will be preserved by default.</td>
+<td>If the user wants to preserve some number of segments from being compacted then he can set this configuration. Example: carbon.numberof.preserve.segments = 2 then 2 latest segments will always be excluded from the compaction. No segments will be preserved by default.<strong>NOTE:</strong> This configuration is useful when the chances of input data can be wrong due to environment scenarios.Preserving some of the latest segments from being compacted can help to easily delete the wrongly loaded segments.Once compacted,it becomes more difficult to determine the exact data to be deleted(except when data is incrementing according to time)</td>
 </tr>
 <tr>
 <td>carbon.allowed.compaction.days</td>
 <td>0</td>
-<td>Compaction will merge the segments which are loaded with in the specific number of days configured. Example: If the configuration is 2, then the segments which are loaded in the time frame of 2 days only will get merged. Segments which are loaded 2 days apart will not be merged. This is disabled by default.</td>
+<td>This configuration is used to control on the number of recent segments that needs to be compacted, ignoring the older ones.This congifuration is in days.For Example: If the configuration is 2, then the segments which are loaded in the time frame of past 2 days only will get merged. Segments which are loaded earlier than 2 days will not be merged. This configuration is disabled by default.<strong>NOTE:</strong> This configuration is useful when a bulk of history data is loaded into the carbondata.Query on this data is less frequent.In such cases involving these segments also into compacation will affect the resource consumption, increases overall compaction time.</td>
 </tr>
 <tr>
 <td>carbon.enable.auto.load.merge</td>
 <td>false</td>
-<td>To enable compaction while data loading.</td>
+<td>Compaction can be automatically triggered once data load completes.This ensures that the segments are merged in time and thus query times doesnt increase with increase in segments.This configuration enables to do compaction along with data loading.**NOTE: **Compaction will be triggered once the data load completes.But the status of data load wait till the compaction is completed.Hence it might look like data loading time has increased, but thats not the case.Moreover failure of compaction will not affect the data loading status.If data load had completed successfully, the status would be updated and segments are committed.However, failure while data loading, will not trigger compaction and error is returned immediately.</td>
 </tr>
 <tr>
 <td>carbon.enable.page.level.reader.in.compaction</td>
 <td>true</td>
-<td>Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory.</td>
+<td>Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory.<strong>NOTE:</strong> Please refer to <a href="./file-structure-of-carbondata.html">file-structure-of-carbondata</a> to understand the storage format of CarbonData and concepts of pages.</td>
+</tr>
+<tr>
+<td>carbon.concurrent.compaction</td>
+<td>true</td>
+<td>Compaction of different tables can be executed concurrently.This configuration determines whether to compact all qualifying tables in parallel or not.**NOTE: **Compacting concurrently is a resource demanding operation and needs more resouces there by affecting the query performance also.This configuration is <strong>deprecated</strong> and might be removed in future releases.</td>
+</tr>
+<tr>
+<td>carbon.compaction.prefetch.enable</td>
+<td>false</td>
+<td>Compaction operation is similar to Query + data load where in data from qualifying segments are queried and data loading performed to generate a new single segment.This configuration determines whether to query ahead data from segments and feed it for data loading.**NOTE: **This configuration is disabled by default as it needs extra resources for querying ahead extra data.Based on the memory availability on the cluster, user can enable it to improve compaction performance.</td>
+</tr>
+<tr>
+<td>carbon.merge.index.in.segment</td>
+<td>true</td>
+<td>Each CarbonData file has a companion CarbonIndex file which maintains the metadata about the data.These CarbonIndex files are read and loaded into driver and is used subsequently for pruning of data during queries.These CarbonIndex files are very small in size(few KB) and are many.Reading many small files from HDFS is not efficient and leads to slow IO performance.Hence these CarbonIndex files belonging to a segment can be combined into  a single file and read once there by increasing the IO throughput.This configuration enables to merge all the CarbonIndex files into a single MergeIndex file upon data loading completion.<strong>NOTE:</strong> Reading a single big file is more efficient in HDFS and IO throughput is very high.Due to this the time needed to load the index files into memory when query is received for the first time on that table is significantly reduced and there by significantly reduces the delay in serving the first query.</td>
 </tr>
 </tbody>
 </table>
-<ul>
-<li><strong>Query Configuration</strong></li>
-</ul>
+<h2>
+<a id="query-configuration" class="anchor" href="#query-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Query Configuration</h2>
 <table>
 <thead>
 <tr>
@@ -633,6 +626,16 @@
 </thead>
 <tbody>
 <tr>
+<td>carbon.max.driver.lru.cache.size</td>
+<td>-1</td>
+<td>Maximum memory <strong>(in MB)</strong> upto which the driver process can cache the data (BTree and dictionary values). Beyond this, least recently used data will be removed from cache before loading new set of values.Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.<strong>NOTE:</strong> Minimum number of entries that needs to be removed from cache in order to load the new set of data is determined and unloaded.ie.,for example if 3 cache entries qualify for pre-emption, out of these, those entries that free up more cache memory is removed prior to others.</td>
+</tr>
+<tr>
+<td>carbon.max.executor.lru.cache.size</td>
+<td>-1</td>
+<td>Maximum memory <strong>(in MB)</strong> upto which the executor process can cache the data (BTree and reverse dictionary values).Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.<strong>NOTE:</strong> If this parameter is not configured, then the value of <em><strong>carbon.max.driver.lru.cache.size</strong></em> will be used.</td>
+</tr>
+<tr>
 <td>max.query.execution.time</td>
 <td>60</td>
 <td>Maximum time allowed for one query to be executed. The value is in minutes.</td>
@@ -640,12 +643,12 @@
 <tr>
 <td>carbon.enableMinMax</td>
 <td>true</td>
-<td>Min max is feature added to enhance query performance. To disable this feature, set it false.</td>
+<td>CarbonData maintains the metadata which enables to prune unnecessary files from being scanned as per the query conditions.To achieve pruning, Min,Max of each column is maintined.Based on the filter condition in the query, certain data can be skipped from scanning by matching the filter value against the min,max values of the column(s) present in that carbondata file.This pruing enhances query performance significantly.</td>
 </tr>
 <tr>
 <td>carbon.dynamicallocation.schedulertimeout</td>
 <td>5</td>
-<td>Specifies the maximum time (unit in seconds) the scheduler can wait for executor to be active. Minimum value is 5 sec and maximum value is 15 sec.</td>
+<td>CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData.To determine the number of tasks that can be scheduled, knowing the count of active executors is necessary.When dynamic allocation is enabled on a YARN based spark cluster,execuor processes are shutdown if no request is received for a particular amount of time.The executors are brought up when the requet is received again.This configuration specifies the maximum time (unit in seconds) the carbon scheduler can wait for executor to be active. Minimum value is 5 sec and maximum value is 15 sec.**NOTE: **Waiting for longer time leads to slow query response time.Moreover it might be possible that YARN is not able to start the executors and waiting is not beneficial.</td>
 </tr>
 <tr>
 <td>carbon.scheduler.minregisteredresourcesratio</td>
@@ -657,35 +660,76 @@
 <td>false</td>
 <td>If set to true, it will use CarbonReader to do distributed scan directly instead of using compute framework like spark, thus avoiding limitation of compute framework like SQL optimizer and task scheduling overhead.</td>
 </tr>
-</tbody>
-</table>
-<ul>
-<li><strong>Global Dictionary Configurations</strong></li>
-</ul>
-<table>
-<thead>
 <tr>
-<th>Parameter</th>
-<th>Default Value</th>
-<th>Description</th>
+<td>carbon.search.query.timeout</td>
+<td>10s</td>
+<td>Time within which the result is expected from the workers;beyond which the query is terminated</td>
 </tr>
-</thead>
-<tbody>
 <tr>
-<td>carbon.cutOffTimestamp</td>
-<td></td>
-<td>Sets the start date for calculating the timestamp. Java counts the number of milliseconds from start of "1970-01-01 00:00:00". This property is used to customize the start of position. For example "2000-01-01 00:00:00". The date must be in the form "carbon.timestamp.format".</td>
+<td>carbon.search.scan.thread</td>
+<td>num of cores available in worker node</td>
+<td>Number of cores to be used in each worker for performing scan.</td>
 </tr>
 <tr>
-<td>carbon.timegranularity</td>
-<td>SECOND</td>
-<td>The property used to set the data granularity level DAY, HOUR, MINUTE, or SECOND.</td>
+<td>carbon.search.master.port</td>
+<td>10020</td>
+<td>Port on which the search master listens for incoming query requests</td>
+</tr>
+<tr>
+<td>carbon.search.worker.port</td>
+<td>10021</td>
+<td>Port on which search master communicates with the workers.</td>
+</tr>
+<tr>
+<td>carbon.search.worker.workload.limit</td>
+<td>10 * <em>carbon.search.scan.thread</em>
+</td>
+<td>Maximum number of active requests that can be sent to a worker.Beyond which the request needs to be rescheduled for later time or to a different worker.</td>
+</tr>
+<tr>
+<td>carbon.detail.batch.size</td>
+<td>100</td>
+<td>The buffer size to store records, returned from the block scan. In limit scenario this parameter is very important. For example your query limit is 1000. But if we set this value to 3000 that means we get 3000 records from scan but spark will only take 1000 rows. So the 2000 remaining are useless. In one Finance test case after we set it to 100, in the limit 1000 scenario the performance increase about 2 times in comparison to if we set this value to 12000.</td>
+</tr>
+<tr>
+<td>carbon.enable.vector.reader</td>
+<td>true</td>
+<td>Spark added vector processing to optimize cpu cache miss and there by increase the query performance.This configuration enables to fetch data as columnar batch of size 4*1024 rows instead of fetching data row by row and provide it to spark so that there is improvement in  select queries performance.</td>
+</tr>
+<tr>
+<td>carbon.task.distribution</td>
+<td>block</td>
+<td>CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData.Each of these task distribution suggestions has its own advantages and disadvantages.Based on the customer use case, appropriate task distribution can be configured.<strong>block</strong>: Setting this value will launch one task per block. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. <strong>custom</strong>: Setting this value will group the blocks and distribute it uniformly to the available resources in the cluster. This enhances the query performance but not suggested in case of concurrent queries and queries having big shuffling scenarios. <strong>blocklet</strong>: Setting this value will launch one task per blocklet. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. <strong>merge_smal
 l_files</strong>: Setting this value will merge all the small carbondata files upto a bigger size configured by <em><strong>spark.sql.files.maxPartitionBytes</strong></em> (128 MB is the default value,it is configurable) during querying. The small carbondata files are combined to a map task to reduce the number of read task. This enhances the performance.</td>
+</tr>
+<tr>
+<td>carbon.custom.block.distribution</td>
+<td>false</td>
+<td>CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData.When this configuration is true, CarbonData would distribute the available blocks to be scanned among the available number of cores.For Example:If there are 10 blocks to be scanned and only 3 tasks can be run(only 3 executor cores available in the cluster), CarbonData would combine blocks as 4,3,3 and give it to 3 tasks to run.<strong>NOTE:</strong> When this configuration is false, as per the <em><strong>carbon.task.distribution</strong></em> configuration, each block/blocklet would be given to each task.</td>
+</tr>
+<tr>
+<td>enable.query.statistics</td>
+<td>false</td>
+<td>CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues.This configuration when made <em><strong>true</strong></em> would log additional query statistics information to more accurately locate the issues being debugged.<strong>NOTE:</strong> Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time.It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately.Also extensive logging is an increased IO operation and hence over all query performance might get reduced.Therefore it is recommened to enable this configuration only for the duration of debugging.</td>
+</tr>
+<tr>
+<td>enable.unsafe.in.query.processing</td>
+<td>true</td>
+<td>CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations.This configuration enables to use unsafe functions in CarbonData while scanning the  data during query.</td>
+</tr>
+<tr>
+<td>carbon.query.validate.directqueryondatamap</td>
+<td>true</td>
+<td>CarbonData supports creating pre-aggregate table datamaps as an independent tables.For some debugging purposes, it might be required to directly query from such datamap tables.This configuration allows to query on such datamaps.</td>
+</tr>
+<tr>
+<td>carbon.heap.memory.pooling.threshold.bytes</td>
+<td>1048576</td>
+<td>CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations.Using unsafe, memory can be allocated on Java Heap or off heap.This configuration controlls the allocation mechanism on Java HEAP.If the heap memory allocations of the given size is greater or equal than this value,it should go through the pooling mechanism.But if set this size to -1, it should not go through the pooling mechanism.Default value is 1048576(1MB, the same as Spark).Value to be specified in bytes.</td>
 </tr>
 </tbody>
 </table>
 <h2>
-<a id="spark-configuration" class="anchor" href="#spark-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Spark Configuration</h2>
-<p><b></b></p><p align="center">Spark Configuration Reference in spark-defaults.conf</p>
+<a id="data-mutation-configuration" class="anchor" href="#data-mutation-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Mutation Configuration</h2>
 <table>
 <thead>
 <tr>
@@ -696,14 +740,24 @@
 </thead>
 <tbody>
 <tr>
-<td>spark.driver.memory</td>
-<td>1g</td>
-<td>Amount of memory to be used by the driver process.</td>
+<td>carbon.insert.persist.enable</td>
+<td>false</td>
+<td>CarbonData does loading in 2 major steps.1st step reads from the input source and generates the dictionary values.2nd step reads from the source again and encodes the data with the dictionary values, perform index calculations and writes in CarbonData format. Suppose we are loading the CarbonData table using another table as source(using insert into) and the source table is being loaded in parallel, there can be cases where some data got inserted into the source table after CarbonData generated for the target table in which case some new records which does not have dictionary values generated gets read leading to inconsistency. To avoid this condition we can persist the dataset of RDD/dataframe into MEMORY_AND_DISK(default value) and perform insert into operation. This ensures the data read from source table is cached and is not read again from the source there by ensuring consistency between dictionary generation and writing to CarbonData format steps. By default this value is 
 false as concurrent loading into source table is not the scenario majority of the times.<strong>NOTE:</strong> This configuration can reduce the insert into execution time as data need not be re read; but increases the memory foot print.</td>
 </tr>
 <tr>
-<td>spark.executor.memory</td>
-<td>1g</td>
-<td>Amount of memory to be used per executor process.</td>
+<td>carbon.insert.storage.level</td>
+<td>MEMORY_AND_DISK</td>
+<td>Storage level to persist dataset of a RDD/dataframe.Applicable when <em><strong>carbon.insert.persist.enable</strong></em> is <strong>true</strong>, if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. <a href="http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence" rel="nofollow">See detail</a>.</td>
+</tr>
+<tr>
+<td>carbon.update.persist.enable</td>
+<td>true</td>
+<td>Configuration to enable the dataset of RDD/dataframe to persist data. Enabling this will reduce the execution time of UPDATE operation.</td>
+</tr>
+<tr>
+<td>carbon.update.storage.level</td>
+<td>MEMORY_AND_DISK</td>
+<td>Storage level to persist dataset of a RDD/dataframe.Applicable when <em><strong>carbon.update.persist.enable</strong></em> is <strong>true</strong>, if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. <a href="http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence" rel="nofollow">See detail</a>.</td>
 </tr>
 </tbody>
 </table>
@@ -767,6 +821,10 @@
 <tbody>
 <tr>
 <td>carbon.options.bad.records.logger.enable</td>
+<td>CarbonData can identify the records that are not conformant to schema and isolate them as bad records.Enabling this configuration will make CarbonData to log such bad records.<strong>NOTE:</strong> If the input data contains many bad records, logging them will slow down the over all data loading throughput.The data load operation status would depend on the configuration in <em><strong>carbon.bad.records.action</strong></em>.</td>
+</tr>
+<tr>
+<td>carbon.options.bad.records.logger.enable</td>
 <td>To enable or disable bad record logger.</td>
 </tr>
 <tr>
@@ -783,7 +841,7 @@
 </tr>
 <tr>
 <td>carbon.options.single.pass</td>
-<td>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.</td>
+<td>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.<strong>NOTE:</strong> Enabling this starts a new dictionary server to handle dictionary generation requests during data loading.Without this option, the input csv files will have to read twice.Once while dictionary generation and persisting to the dictionary files.second when the data loading need to convert the input data into carbondata format.Enabling this optimizes the optimizes to read the input data only once there by reducing IO and hence over all data loading time.If concurrent data loading needs to be supported, consider tuning <em><strong>dictionary.worker.threads</strong></em>.Port on which the dictionary server
  need to listen on can be configured using the configuration <em><strong>carbon.dictionary.server.port</strong></em>.</td>
 </tr>
 <tr>
 <td>carbon.options.bad.record.path</td>
@@ -791,12 +849,36 @@
 </tr>
 <tr>
 <td>carbon.custom.block.distribution</td>
-<td>Specifies whether to use the Spark or Carbon block distribution feature.</td>
+<td>Specifies whether to use the Spark or Carbon block distribution feature.**NOTE: **Refer to <a href="#query-configuration">Query Configuration</a>#carbon.custom.block.distribution for more details on CarbonData scheduler.</td>
 </tr>
 <tr>
 <td>enable.unsafe.sort</td>
 <td>Specifies whether to use unsafe sort during data loading. Unsafe sort reduces the garbage collection during data load operation, resulting in better performance.</td>
 </tr>
+<tr>
+<td>carbon.options.dateformat</td>
+<td>Specifies the data format of the date columns in the data being loaded</td>
+</tr>
+<tr>
+<td>carbon.options.timestampformat</td>
+<td>Specifies the timestamp format of the time stamp columns in the data being loaded</td>
+</tr>
+<tr>
+<td>carbon.options.sort.scope</td>
+<td>Specifies how the current data load should be sorted with.**NOTE: **Refer to <a href="#data-loading-configuration">Data Loading Configuration</a>#carbon.sort.scope for detailed information.</td>
+</tr>
+<tr>
+<td>carbon.options.global.sort.partitions</td>
+<td></td>
+</tr>
+<tr>
+<td>carbon.options.serialization.null.format</td>
+<td>Default Null value representation in the data being loaded.<strong>NOTE:</strong> Refer to <a href="#data-loading-configuration">Data Loading Configuration</a>#carbon.options.serialization.null.format for detailed information.</td>
+</tr>
+<tr>
+<td>carbon.query.directQueryOnDataMap.enabled</td>
+<td>Specifies whether datamap can be queried directly.This is useful for debugging purposes.**NOTE: **Refer to <a href="#query-configuration">Query Configuration</a>#carbon.query.validate.directqueryondatamap for detailed information.</td>
+</tr>
 </tbody>
 </table>
 <p><strong>Examples:</strong></p>
@@ -824,6 +906,17 @@
 <p>Failure will be displayed in the UI.</p>
 </li>
 </ul>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -839,4 +932,4 @@
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/css/style.css
----------------------------------------------------------------------
diff --git a/src/main/webapp/css/style.css b/src/main/webapp/css/style.css
index 94b8fbf..88fd05f 100644
--- a/src/main/webapp/css/style.css
+++ b/src/main/webapp/css/style.css
@@ -4,7 +4,7 @@
 
 *{margin: 0; padding: 0;}
 
-body{background:#fdfdfd; color:#000 ; font-family:'Open Sans', arial, helvatica,  sans-serif; font-size:14px; line-height:22px;}
+body{background:#fdfdfd; color:#000 ; font-family:'Helvetica Neue', arial, helvatica,  sans-serif; font-size:14px; line-height:22px;}
 
 a{-webkit-transition: all .1s ease-in-out;
    -moz-transition: all .1s ease-in-out;
@@ -1301,3 +1301,77 @@ box-shadow: 0 3px 2px #aaa }
     white-space: nowrap;
 }
 
+.mdcontent {
+float:right;
+width:80%;
+padding-left:30px;
+}
+
+.verticalnavbar {
+    float: left;
+    text-transform: uppercase;
+    width: 15%;
+    font-family:"Helvetica Neue";
+    padding-top: 90px;
+    position: fixed;
+}
+.nav__item,
+.nav__item__with__subs {
+    color: #000000;
+    border-right: 2px solid #000000;
+    display: block;
+    padding-top: 1.5rem;
+    position: relative;
+}
+.nav__item__with__subs {
+    padding-top: 0;
+}
+.nav__sub__anchor,
+.nav__sub__item {
+    border-right: none;
+}
+.nav__sub__item {
+    display: none;
+    color: #888888;
+    font-size: 1.2rem;
+    text-transform: capitalize;
+}
+.nav__item__with__subs--expanded .nav__sub__item {
+    display: block;
+}
+.nav__item:first-of-type {
+    padding-top: 0;
+}
+.nav__item__with__subs .nav__item:first-of-type {
+    padding-top: 1.5rem;
+}
+.nav__item::after {
+    content: "";
+    display: block;
+    height: 2.1rem;
+    width: 1.1rem;
+    border-radius: 1rem;
+    -moz-border-radius: 1rem;
+    -webkit-border-radius: 1rem;
+    border: 2px solid #000000;
+    background: #FFFFFF;
+    position: absolute;
+    right: -.7rem;
+    top: 1.7rem;
+    opacity: 0;
+    transition: opacity .2s ease-out;
+}
+.nav__item.selected::after {
+    opacity: 1;
+}
+.nav__item.selected:first-of-type::after {
+    top: .4rem;
+}
+.nav__item__with__subs .nav__item:first-of-type::after {
+    top: 1.7rem;
+}
+verticalnavbar .btn {
+    display: block;
+    margin-top: 4rem;
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/data-management-on-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/data-management-on-carbondata.html b/src/main/webapp/data-management-on-carbondata.html
index 566bb8e..bb5ae78 100644
--- a/src/main/webapp/data-management-on-carbondata.html
+++ b/src/main/webapp/data-management-on-carbondata.html
@@ -1318,4 +1318,4 @@ Future {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>


[35/39] carbondata-site git commit: refactore document

Posted by ch...@apache.org.
refactore document


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

Branch: refs/heads/asf-site
Commit: ca81b940b86abe79c2a157503d7508489563719a
Parents: 324588f a51dc59
Author: chenliang613 <ch...@huawei.com>
Authored: Sat Sep 8 00:52:49 2018 +0800
Committer: chenliang613 <ch...@huawei.com>
Committed: Sat Sep 8 00:52:49 2018 +0800

----------------------------------------------------------------------
 .../classes/CleanUp$$anonfun$cleanUp$1.class    |  Bin 0 -> 1048 bytes
 .../classes/MDFileConverter$$anonfun$1.class    |  Bin 0 -> 1606 bytes
 .../classes/MDFileConverter$$anonfun$2.class    |  Bin 0 -> 1617 bytes
 content/WEB-INF/classes/MDFileConverter.class   |  Bin 10321 -> 10608 bytes
 ...ler$$anonfun$convertReadMeExtensions$1.class |  Bin 0 -> 2104 bytes
 content/WEB-INF/classes/MdFileHandler.class     |  Bin 6144 -> 6285 bytes
 ...eLauncher$$anonfun$deleteRecursively$1.class |  Bin 0 -> 1133 bytes
 content/WEB-INF/classes/WebsiteLauncher$.class  |  Bin 4209 -> 4248 bytes
 content/WEB-INF/classes/application.conf        |   37 +-
 content/WEB-INF/classes/html/header.html        |   55 +-
 content/bloomfilter-datamap-guide.html          |   69 +-
 content/configuration-parameters.html           |  706 +++++----
 content/css/style.css                           |   76 +-
 content/data-management-on-carbondata.html      | 1321 -----------------
 content/data-management.html                    |  413 ------
 content/datamap-developer-guide.html            |   76 +-
 content/datamap-management.html                 |   85 +-
 content/ddl-of-carbondata.html                  | 1167 +++++++++++++++
 content/ddl-operation-on-carbondata.html        |  748 ----------
 content/dml-of-carbondata.html                  |  681 +++++++++
 content/dml-operation-on-carbondata.html        |  716 ---------
 content/documentation.html                      |  252 ++++
 content/faq.html                                |  275 +++-
 content/file-structure-of-carbondata.html       |  211 ++-
 .../how-to-contribute-to-apache-carbondata.html |  370 +++++
 content/index.html                              |    4 +-
 content/installation-guide.html                 |  455 ------
 content/introduction.html                       |  367 +++++
 content/language-manual.html                    |  271 ++++
 content/lucene-datamap-guide.html               |   71 +-
 content/mainpage.html                           |  214 ---
 content/pdf/maven-pdf-plugin.pdf                |  Bin 216771 -> 155540 bytes
 content/performance-tuning.html                 |  591 ++++++++
 content/preaggregate-datamap-guide.html         |   73 +-
 content/quick-start-guide.html                  |  497 ++++++-
 content/release-guide.html                      |   87 +-
 content/s3-guide.html                           |   80 +-
 content/sdk-guide.html                          |  125 +-
 content/sdk-writer-guide.html                   |  549 -------
 content/segment-management-on-carbondata.html   |  325 ++++
 content/streaming-guide.html                    |  241 ++-
 content/supported-data-types-in-carbondata.html |   69 +-
 content/timeseries-datamap-guide.html           |  102 +-
 content/troubleshooting.html                    |  366 -----
 content/usecases.html                           |  619 ++++++++
 content/useful-tips-on-carbondata.html          |  480 ------
 src/main/resources/application.conf             |   37 +-
 src/main/scala/MDFileConverter.scala            |    6 +-
 src/main/scala/MdFileHandler.scala              |    6 +-
 src/main/scala/html/header.html                 |   55 +-
 .../scala/scripts/bloomfilter-datamap-guide     |   11 +
 src/main/scala/scripts/configuration-parameters |   11 +
 src/main/scala/scripts/datamap-developer-guide  |   11 +
 src/main/scala/scripts/datamap-management       |   11 +
 src/main/scala/scripts/ddl-of-carbondata        |   11 +
 src/main/scala/scripts/dml-of-carbondata        |   11 +
 src/main/scala/scripts/faq                      |    4 +
 .../scala/scripts/file-structure-of-carbondata  |   11 +
 .../how-to-contribute-to-apache-carbondata      |    4 +
 src/main/scala/scripts/introduction             |    4 +
 src/main/scala/scripts/language-manual          |   11 +
 src/main/scala/scripts/lucene-datamap-guide     |   11 +
 src/main/scala/scripts/performance-tuning       |    4 +
 .../scala/scripts/preaggregate-datamap-guide    |   11 +
 src/main/scala/scripts/quick-start-guide        |    4 +
 src/main/scala/scripts/release-guide            |    4 +
 src/main/scala/scripts/s3-guide                 |    4 +
 src/main/scala/scripts/sdk-guide                |    4 +
 .../scripts/segment-management-on-carbondata    |   10 +
 src/main/scala/scripts/streaming-guide          |   11 +
 .../scripts/supported-data-types-in-carbondata  |   11 +
 src/main/scala/scripts/timeseries-datamap-guide |   11 +
 src/main/scala/scripts/usecases                 |    4 +
 src/main/webapp/bloomfilter-datamap-guide.html  |   69 +-
 src/main/webapp/configuration-parameters.html   |  706 +++++----
 src/main/webapp/css/style.css                   |   76 +-
 .../webapp/data-management-on-carbondata.html   | 1321 -----------------
 src/main/webapp/datamap-developer-guide.html    |   76 +-
 src/main/webapp/datamap-management.html         |   85 +-
 src/main/webapp/ddl-of-carbondata.html          | 1167 +++++++++++++++
 src/main/webapp/dml-of-carbondata.html          |  681 +++++++++
 src/main/webapp/documentation.html              |  252 ++++
 src/main/webapp/faq.html                        |  275 +++-
 .../webapp/file-structure-of-carbondata.html    |  211 ++-
 .../how-to-contribute-to-apache-carbondata.html |  370 +++++
 src/main/webapp/index.html                      |    4 +-
 src/main/webapp/installation-guide.html         |  455 ------
 src/main/webapp/introduction.html               |  367 +++++
 src/main/webapp/language-manual.html            |  271 ++++
 src/main/webapp/lucene-datamap-guide.html       |   71 +-
 src/main/webapp/mainpage.html                   |  214 ---
 src/main/webapp/performance-tuning.html         |  591 ++++++++
 src/main/webapp/preaggregate-datamap-guide.html |   73 +-
 src/main/webapp/quick-start-guide.html          |  497 ++++++-
 src/main/webapp/release-guide.html              |  571 +++++++
 src/main/webapp/s3-guide.html                   |   80 +-
 src/main/webapp/sdk-guide.html                  |  125 +-
 src/main/webapp/sdk-writer-guide.html           |  549 -------
 .../segment-management-on-carbondata.html       |  325 ++++
 src/main/webapp/streaming-guide.html            |  241 ++-
 .../supported-data-types-in-carbondata.html     |   69 +-
 src/main/webapp/timeseries-datamap-guide.html   |  102 +-
 src/main/webapp/troubleshooting.html            |  366 -----
 src/main/webapp/usecases.html                   |  619 ++++++++
 src/main/webapp/useful-tips-on-carbondata.html  |  480 ------
 src/site/markdown/bloomfilter-datamap-guide.md  |    5 +-
 src/site/markdown/configuration-parameters.md   |  273 ++--
 .../markdown/data-management-on-carbondata.md   | 1394 ------------------
 src/site/markdown/datamap-developer-guide.md    |   13 +-
 src/site/markdown/datamap-management.md         |   17 +-
 src/site/markdown/ddl-of-carbondata.md          |  957 ++++++++++++
 src/site/markdown/dml-of-carbondata.md          |  469 ++++++
 src/site/markdown/documentation.md              |   66 +
 src/site/markdown/faq.md                        |  283 +++-
 .../markdown/file-structure-of-carbondata.md    |  178 ++-
 .../how-to-contribute-to-apache-carbondata.md   |  192 +++
 src/site/markdown/installation-guide.md         |  198 ---
 src/site/markdown/introduction.md               |  117 ++
 src/site/markdown/language-manual.md            |   39 +
 src/site/markdown/lucene-datamap-guide.md       |    4 +-
 src/site/markdown/performance-tuning.md         |  246 ++++
 src/site/markdown/preaggregate-datamap-guide.md |    7 +-
 src/site/markdown/quick-start-guide.md          |  387 ++++-
 src/site/markdown/release-guide.md              |  422 ++++++
 src/site/markdown/s3-guide.md                   |   17 +-
 src/site/markdown/sdk-guide.md                  |   82 +-
 .../segment-management-on-carbondata.md         |  142 ++
 src/site/markdown/streaming-guide.md            |  172 ++-
 .../supported-data-types-in-carbondata.md       |    5 +-
 src/site/markdown/timeseries-datamap-guide.md   |   38 +-
 src/site/markdown/troubleshooting.md            |  267 ----
 src/site/markdown/usecases.md                   |  215 +++
 src/site/markdown/useful-tips-on-carbondata.md  |  179 ---
 133 files changed, 19033 insertions(+), 11847 deletions(-)
----------------------------------------------------------------------



[32/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/data-management.html
----------------------------------------------------------------------
diff --git a/content/data-management.html b/content/data-management.html
deleted file mode 100644
index a9086ca..0000000
--- a/content/data-management.html
+++ /dev/null
@@ -1,413 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
-                                   target="_blank">Apache CarbonData 1.2.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
-                                   target="_blank">Apache CarbonData 1.1.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
-                                   target="_blank">Apache CarbonData 1.1.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
-                                   target="_blank">Apache CarbonData 1.0.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
-                                   target="_blank">Apache CarbonData 0.2.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
-                                   target="_blank">Apache CarbonData 0.1.1</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
-                                   target="_blank">Apache CarbonData 0.1.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search" target="_blank">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input"  placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="row">
-                                <div class="col-sm-12  col-md-12">
-                                    <div>
-<h1>
-<a id="data-management" class="anchor" href="#data-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Management</h1>
-<p>This tutorial is going to introduce you to the conceptual details of data management like:</p>
-<ul>
-<li><a href="#loading-data">Loading Data</a></li>
-<li><a href="#deleting-data">Deleting Data</a></li>
-<li><a href="#compacting-data">Compacting Data</a></li>
-<li><a href="#updating-data">Updating Data</a></li>
-</ul>
-<h2>
-<a id="loading-data" class="anchor" href="#loading-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Loading Data</h2>
-<ul>
-<li>
-<p><strong>Scenario</strong></p>
-<p>After creating a table, you can load data to the table using the <a href="dml-operation-on-carbondata.html">LOAD DATA</a> command. The loaded data is available for querying.
-When data load is triggered, the data is encoded in CarbonData format and copied into HDFS CarbonData store path (specified in carbon.properties file)
-in compressed, multi dimensional columnar format for quick analysis queries. The same command can be used to load new data or to
-update the existing data. Only one data load can be triggered for one table. The high cardinality columns of the dictionary encoding are
-automatically recognized and these columns will not be used for dictionary encoding.</p>
-</li>
-<li>
-<p><strong>Procedure</strong></p>
-<p>Data loading is a process that involves execution of multiple steps to read, sort and encode the data in CarbonData store format.
-Each step is executed on different threads. After data loading process is complete, the status (success/partial success) is updated to
-CarbonData store metadata. The table below lists the possible load status.</p>
-</li>
-</ul>
-<table>
-<thead>
-<tr>
-<th>Status</th>
-<th>Description</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>Success</td>
-<td>All the data is loaded into table and no bad records found.</td>
-</tr>
-<tr>
-<td>Partial Success</td>
-<td>Data is loaded into table and bad records are found. Bad records are stored at carbon.badrecords.location.</td>
-</tr>
-</tbody>
-</table>
-<p>In case of failure, the error will be logged in error log. Details of loads can be seen with <a href="dml-operation-on-carbondata.html#show-segments">SHOW SEGMENTS</a> command. The show segment command output consists of :</p>
-<ul>
-<li>SegmentSequenceId</li>
-<li>Status</li>
-<li>Load Start Time</li>
-<li>Load End Time</li>
-</ul>
-<p>The latest load will be displayed first in the output.</p>
-<p>Refer to <a href="dml-operation-on-carbondata.html">DML operations on CarbonData</a> for load commands.</p>
-<h2>
-<a id="deleting-data" class="anchor" href="#deleting-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deleting Data</h2>
-<ul>
-<li>
-<p><strong>Scenario</strong></p>
-<p>If you have loaded wrong data into the table, or too many bad records are present and you want to modify and reload the data, you can delete required data loads.
-The load can be deleted using the Segment Sequence Id or if the table contains date field then the data can be deleted using the date field.
-If there are some specific records that need to be deleted based on some filter condition(s) we can delete by records.</p>
-</li>
-<li>
-<p><strong>Procedure</strong></p>
-<p>The loaded data can be deleted in the following ways:</p>
-<ul>
-<li>
-<p>Delete by Segment ID</p>
-<p>After you get the segment ID of the segment that you want to delete, execute the delete command for the selected segment.
-The status of deleted segment is updated to Marked for delete / Marked for Update.</p>
-</li>
-</ul>
-</li>
-</ul>
-<table>
-<thead>
-<tr>
-<th>SegmentSequenceId</th>
-<th>Status</th>
-<th>Load Start Time</th>
-<th>Load End Time</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>0</td>
-<td>Success</td>
-<td>2015-11-19 19:14:...</td>
-<td>2015-11-19 19:14:...</td>
-</tr>
-<tr>
-<td>1</td>
-<td>Marked for Update</td>
-<td>2015-11-19 19:54:...</td>
-<td>2015-11-19 20:08:...</td>
-</tr>
-<tr>
-<td>2</td>
-<td>Marked for Delete</td>
-<td>2015-11-19 20:25:...</td>
-<td>2015-11-19 20:49:...</td>
-</tr>
-</tbody>
-</table>
-<ul>
-<li>
-<p>Delete by Date Field</p>
-<p>If the table contains date field, you can delete the data based on a specific date.</p>
-</li>
-<li>
-<p>Delete by Record</p>
-<p>To delete records from CarbonData table based on some filter Condition(s).</p>
-<p>For delete commands refer to <a href="dml-operation-on-carbondata.html">DML operations on CarbonData</a>.</p>
-</li>
-<li>
-<p><strong>NOTE</strong>:</p>
-<ul>
-<li>
-<p>When the delete segment DML is called, segment will not be deleted physically from the file system. Instead the segment status will be marked as "Marked for Delete". For the query execution, this deleted segment will be excluded.</p>
-</li>
-<li>
-<p>The deleted segment will be deleted physically during the next load operation and only after the maximum query execution time configured using "max.query.execution.time". By default it is 60 minutes.</p>
-</li>
-<li>
-<p>If the user wants to force delete the segment physically then he can use CLEAN FILES Command.</p>
-</li>
-</ul>
-</li>
-</ul>
-<p>Example :</p>
-<pre><code>CLEAN FILES FOR TABLE table1
-</code></pre>
-<p>This DML will physically delete the segment which are "Marked for delete" and "Compacted" immediately.</p>
-<h2>
-<a id="compacting-data" class="anchor" href="#compacting-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Compacting Data</h2>
-<ul>
-<li>
-<p><strong>Scenario</strong></p>
-<p>Frequent data ingestion results in several fragmented CarbonData files in the store directory. Since data is sorted only within each load, the indices perform only within each
-load. This means that there will be one index for each load and as number of data load increases, the number of indices also increases. As each index works only on one load,
-the performance of indices is reduced. CarbonData provides provision for compacting the loads. Compaction process combines several segments into one large segment by merge sorting the data from across the segments.</p>
-</li>
-<li>
-<p><strong>Procedure</strong></p>
-<p>There are two types of compaction Minor and Major compaction.</p>
-<ul>
-<li>
-<p><strong>Minor Compaction</strong></p>
-<p>In minor compaction the user can specify how many loads to be merged. Minor compaction triggers for every data load if the parameter carbon.enable.auto.load.merge is set. If any segments are available to be merged, then compaction will
-run parallel with data load. There are 2 levels in minor compaction.</p>
-<ul>
-<li>Level 1: Merging of the segments which are not yet compacted.</li>
-<li>Level 2: Merging of the compacted segments again to form a bigger segment.</li>
-</ul>
-</li>
-<li>
-<p><strong>Major Compaction</strong></p>
-<p>In Major compaction, many segments can be merged into one big segment. User will specify the compaction size until which segments can be merged. Major compaction is usually done during the off-peak time.</p>
-</li>
-</ul>
-<p>There are number of parameters related to Compaction that can be set in carbon.properties file</p>
-</li>
-</ul>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Default</th>
-<th>Application</th>
-<th>Description</th>
-<th>Valid Values</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>carbon.compaction.level.threshold</td>
-<td>4, 3</td>
-<td>Minor</td>
-<td>This property is for minor compaction which decides how many segments to be merged. Example: If it is set as "2, 3", then minor compaction will be triggered for every 2 segments in level 1. 3 is the number of level 1 compacted segment which is further compacted to new segment in level 2.</td>
-<td>NA</td>
-</tr>
-<tr>
-<td>carbon.major.compaction.size</td>
-<td>1024 MB</td>
-<td>Major</td>
-<td>Major compaction size can be configured using this parameter. Sum of the segments which is below this threshold will be merged.</td>
-<td>NA</td>
-</tr>
-<tr>
-<td>carbon.numberof.preserve.segments</td>
-<td>0</td>
-<td>Minor/Major</td>
-<td>This property configures number of segments to preserve from being compacted. Example: carbon.numberof.preserve.segments=2 then 2 latest segments will always be excluded from the compaction. No segments will be preserved by default.</td>
-<td>0-100</td>
-</tr>
-<tr>
-<td>carbon.allowed.compaction.days</td>
-<td>0</td>
-<td>Minor/Major</td>
-<td>Compaction will merge the segments which are loaded within the specific number of days configured. Example: If the configuration is 2, then the segments which are loaded in the time frame of 2 days only will get merged. Segments which are loaded 2 days apart will not be merged. This is disabled by default.</td>
-<td>0-100</td>
-</tr>
-<tr>
-<td>carbon.number.of.cores.while.compacting</td>
-<td>2</td>
-<td>Minor/Major</td>
-<td>Number of cores which is used to write data during compaction.</td>
-<td>0-100</td>
-</tr>
-</tbody>
-</table>
-<p>For compaction commands refer to <a href="ddl-operation-on-carbondata.html">DDL operations on CarbonData</a></p>
-<h2>
-<a id="updating-data" class="anchor" href="#updating-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Updating Data</h2>
-<ul>
-<li>
-<p><strong>Scenario</strong></p>
-<p>Sometimes after the data has been ingested into the System, it is required to be updated. Also there may be situations where some specific columns need to be updated
-on the basis of column expression and optional filter conditions.</p>
-</li>
-<li>
-<p><strong>Procedure</strong></p>
-<p>To update we need to specify the column expression with an optional filter condition(s).</p>
-<p>For update commands refer to <a href="dml-operation-on-carbondata.html">DML operations on CarbonData</a>.</p>
-</li>
-</ul>
-</div>
-</div>
-</div>
-</div>
-<div class="doc-footer">
-    <a href="#top" class="scroll-top">Top</a>
-</div>
-</div>
-</section>
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/datamap-developer-guide.html
----------------------------------------------------------------------
diff --git a/content/datamap-developer-guide.html b/content/datamap-developer-guide.html
index 073af96..4b9aa4b 100644
--- a/content/datamap-developer-guide.html
+++ b/content/datamap-developer-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -240,8 +240,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -256,4 +255,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/datamap-management.html
----------------------------------------------------------------------
diff --git a/content/datamap-management.html b/content/datamap-management.html
index 8727594..e2e89f3 100644
--- a/content/datamap-management.html
+++ b/content/datamap-management.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -368,8 +368,7 @@ $(function() {
     $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -384,4 +383,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/ddl-of-carbondata.html
----------------------------------------------------------------------
diff --git a/content/ddl-of-carbondata.html b/content/ddl-of-carbondata.html
index 3e15a83..2582f4d 100644
--- a/content/ddl-of-carbondata.html
+++ b/content/ddl-of-carbondata.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -296,11 +296,101 @@ STORED AS carbondata
 <p><strong>NOTE:</strong> CarbonData also supports "STORED AS carbondata" and "USING carbondata". Find example code at <a href="https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala" target=_blank>CarbonSessionExample</a> in the CarbonData repo.</p>
 <h3>
 <a id="usage-guidelines" class="anchor" href="#usage-guidelines" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
-<p><strong>Supported properties:</strong> <a href="#dictionary-encoding-configuration">DICTIONARY_INCLUDE</a>,<a href="#inverted-index-configuration">NO_INVERTED_INDEX</a>,<a href="#sort-columns-configuration">SORT_COLUMNS</a>,<a href="#sort-scope-configuration">SORT_SCOPE</a>,<a href="#table-block-size-configuration">TABLE_BLOCKSIZE</a>,<a href="#table-compaction-configuration">MAJOR_COMPACTION_SIZE</a>,</p>
-<p><a href="#table-compaction-configuration">AUTO_LOAD_MERGE</a>,<a href="#table-compaction-configuration">COMPACTION_LEVEL_THRESHOLD</a>,<a href="#table-compaction-configuration">COMPACTION_PRESERVE_SEGMENTS</a>,<a href="#table-compaction-configuration">ALLOWED_COMPACTION_DAYS</a>,</p>
-<p><a href="#streaming">streaming</a>,<a href="#local-dictionary-configuration">LOCAL_DICTIONARY_ENABLE</a>,<a href="#local-dictionary-configuration">LOCAL_DICTIONARY_THRESHOLD</a>,<a href="#local-dictionary-configuration">LOCAL_DICTIONARY_INCLUDE</a>,</p>
-<p><a href="#local-dictionary-configuration">LOCAL_DICTIONARY_EXCLUDE</a>,<a href="#caching-minmax-value-for-required-columns">COLUMN_META_CACHE</a>,<a href="#caching-at-block-or-blocklet-level">CACHE_LEVEL</a>,<a href="#support-flat-folder-same-as-hiveparquet">flat_folder</a>,<a href="#string-longer-than-32000-characters">LONG_STRING_COLUMNS</a>,<a href="#bucketing">BUCKETNUMBER</a>,</p>
-<p><a href="#bucketing">BUCKETCOLUMNS</a></p>
+<p><strong>Supported properties:</strong></p>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td><a href="#dictionary-encoding-configuration">DICTIONARY_INCLUDE</a></td>
+<td>Columns for which dictionary needs to be generated</td>
+</tr>
+<tr>
+<td><a href="#inverted-index-configuration">NO_INVERTED_INDEX</a></td>
+<td>Columns to exclude from inverted index generation</td>
+</tr>
+<tr>
+<td><a href="#sort-columns-configuration">SORT_COLUMNS</a></td>
+<td>Columns to include in sort and its order of sort</td>
+</tr>
+<tr>
+<td><a href="#sort-scope-configuration">SORT_SCOPE</a></td>
+<td>Sort scope of the load.Options include no sort, local sort ,batch sort and global sort</td>
+</tr>
+<tr>
+<td><a href="#table-block-size-configuration">TABLE_BLOCKSIZE</a></td>
+<td>Size of blocks to write onto hdfs</td>
+</tr>
+<tr>
+<td><a href="#table-compaction-configuration">MAJOR_COMPACTION_SIZE</a></td>
+<td>Size upto which the segments can be combined into one</td>
+</tr>
+<tr>
+<td><a href="#table-compaction-configuration">AUTO_LOAD_MERGE</a></td>
+<td>Whether to auto compact the segments</td>
+</tr>
+<tr>
+<td><a href="#table-compaction-configuration">COMPACTION_LEVEL_THRESHOLD</a></td>
+<td>Number of segments to compact into one segment</td>
+</tr>
+<tr>
+<td><a href="#table-compaction-configuration">COMPACTION_PRESERVE_SEGMENTS</a></td>
+<td>Number of latest segments that needs to be excluded from compaction</td>
+</tr>
+<tr>
+<td><a href="#table-compaction-configuration">ALLOWED_COMPACTION_DAYS</a></td>
+<td>Segments generated within the configured time limit in days will be compacted, skipping others</td>
+</tr>
+<tr>
+<td><a href="#streaming">streaming</a></td>
+<td>Whether the table is a streaming table</td>
+</tr>
+<tr>
+<td><a href="#local-dictionary-configuration">LOCAL_DICTIONARY_ENABLE</a></td>
+<td>Enable local dictionary generation</td>
+</tr>
+<tr>
+<td><a href="#local-dictionary-configuration">LOCAL_DICTIONARY_THRESHOLD</a></td>
+<td>Cardinality upto which the local dictionary can be generated</td>
+</tr>
+<tr>
+<td><a href="#local-dictionary-configuration">LOCAL_DICTIONARY_INCLUDE</a></td>
+<td>Columns for which local dictionary needs to be generated.Useful when local dictionary need not be generated for all string/varchar/char columns</td>
+</tr>
+<tr>
+<td><a href="#local-dictionary-configuration">LOCAL_DICTIONARY_EXCLUDE</a></td>
+<td>Columns for which local dictionary generation should be skipped.Useful when local dictionary need not be generated for few string/varchar/char columns</td>
+</tr>
+<tr>
+<td><a href="#caching-minmax-value-for-required-columns">COLUMN_META_CACHE</a></td>
+<td>Columns whose metadata can be cached in Driver for efficient pruning and improved query performance</td>
+</tr>
+<tr>
+<td><a href="#caching-at-block-or-blocklet-level">CACHE_LEVEL</a></td>
+<td>Column metadata caching level.Whether to cache column metadata of block or blocklet</td>
+</tr>
+<tr>
+<td><a href="#support-flat-folder-same-as-hiveparquet">flat_folder</a></td>
+<td>Whether to write all the carbondata files in a single folder.Not writing segments folder during incremental load</td>
+</tr>
+<tr>
+<td><a href="#string-longer-than-32000-characters">LONG_STRING_COLUMNS</a></td>
+<td>Columns which are greater than 32K characters</td>
+</tr>
+<tr>
+<td><a href="#bucketing">BUCKETNUMBER</a></td>
+<td>Number of buckets to be created</td>
+</tr>
+<tr>
+<td><a href="#bucketing">BUCKETCOLUMNS</a></td>
+<td>Columns which are to be placed in buckets</td>
+</tr>
+</tbody>
+</table>
 <p>Following are the guidelines for TBLPROPERTIES, CarbonData's additional table options can be set via carbon.properties.</p>
 <ul>
 <li>
@@ -356,15 +446,15 @@ And if you care about loading resources isolation strictly, because the system u
 <pre><code>### Example:
 </code></pre>
 <pre><code> 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'
+   productNumber INT,
+   productName STRING,
+   storeCity STRING,
+   storeProvince STRING,
+   productCategory STRING,
+   productBatch STRING,
+   saleQuantity INT,
+   revenue INT)
+ STORED AS carbondata
  TBLPROPERTIES ('SORT_COLUMNS'='productName,storeCity',
                 'SORT_SCOPE'='NO_SORT')
 </code></pre>
@@ -458,7 +548,7 @@ Following are 5 configurations:</p>
 <tr>
 <td>LOCAL_DICTIONARY_ENABLE</td>
 <td>false</td>
-<td>Whether to enable local dictionary generation. <strong>NOTE:</strong> If this property is defined, it will override the value configured at system level by 'carbon.local.dictionary.enable'</td>
+<td>Whether to enable local dictionary generation. <strong>NOTE:</strong> If this property is defined, it will override the value configured at system level by '<em><strong>carbon.local.dictionary.enable</strong></em>'.Local dictionary will be generated for all string/varchar/char columns unless LOCAL_DICTIONARY_INCLUDE, LOCAL_DICTIONARY_EXCLUDE is configured.</td>
 </tr>
 <tr>
 <td>LOCAL_DICTIONARY_THRESHOLD</td>
@@ -468,12 +558,12 @@ Following are 5 configurations:</p>
 <tr>
 <td>LOCAL_DICTIONARY_INCLUDE</td>
 <td>string/varchar/char columns</td>
-<td>Columns for which Local Dictionary has to be generated.<strong>NOTE:</strong> Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.</td>
+<td>Columns for which Local Dictionary has to be generated.<strong>NOTE:</strong> Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.This property needs to be configured only when local dictionary needs to be generated for few columns, skipping others.This property takes effect only when <strong>LOCAL_DICTIONARY_ENABLE</strong> is true or <strong>carbon.local.dictionary.enable</strong> is true</td>
 </tr>
 <tr>
 <td>LOCAL_DICTIONARY_EXCLUDE</td>
 <td>none</td>
-<td>Columns for which Local Dictionary need not be generated.</td>
+<td>Columns for which Local Dictionary need not be generated.This property needs to be configured only when local dictionary needs to be skipped for few columns, generating for others.This property takes effect only when <strong>LOCAL_DICTIONARY_ENABLE</strong> is true or <strong>carbon.local.dictionary.enable</strong> is true</td>
 </tr>
 </tbody>
 </table>
@@ -502,7 +592,7 @@ Following are 5 configurations:</p>
           
             column3 LONG )
           
-  STORED BY 'carbondata'
+  STORED AS carbondata
   TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='true','LOCAL_DICTIONARY_THRESHOLD'='1000',
   'LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
 </code></pre>
@@ -613,7 +703,7 @@ You can refer to SDKwriterTestCase for example.</p>
 <a id="create-table-as-select" class="anchor" href="#create-table-as-select" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE AS SELECT</h2>
 <p>This function allows user 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.</p>
 <pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name 
-STORED BY 'carbondata' 
+STORED AS carbondata 
 [TBLPROPERTIES (key1=val1, key2=val2, ...)] 
 AS select_statement;
 </code></pre>
@@ -629,7 +719,7 @@ carbon.sql("INSERT INTO source_table SELECT 1,'bob','shenzhen',27")
 carbon.sql("INSERT INTO source_table SELECT 2,'david','shenzhen',31")
 
 carbon.sql("CREATE TABLE target_table
-            STORED BY 'carbondata'
+            STORED AS carbondata
             AS SELECT city,avg(age) FROM source_table GROUP BY city")
             
 carbon.sql("SELECT * FROM target_table").show
@@ -645,21 +735,21 @@ carbon.sql("SELECT * FROM target_table").show
 <a id="create-external-table" class="anchor" href="#create-external-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE EXTERNAL TABLE</h2>
 <p>This function allows user to create external table by specifying location.</p>
 <pre><code>CREATE EXTERNAL TABLE [IF NOT EXISTS] [db_name.]table_name 
-STORED BY 'carbondata' LOCATION ?$FilesPath?
+STORED AS carbondata LOCATION ?$FilesPath?
 </code></pre>
 <h3>
 <a id="create-external-table-on-managed-table-data-location" class="anchor" href="#create-external-table-on-managed-table-data-location" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create external table on managed table data location.</h3>
 <p>Managed table data location provided will have both FACT and Metadata folder.
 This data can be generated by creating a normal carbon table and use this path as $FilesPath in the above syntax.</p>
 <p><strong>Example:</strong></p>
-<pre><code>sql("CREATE TABLE origin(key INT, value STRING) STORED BY 'carbondata'")
+<pre><code>sql("CREATE TABLE origin(key INT, value STRING) STORED AS carbondata")
 sql("INSERT INTO origin select 100,'spark'")
 sql("INSERT INTO origin select 200,'hive'")
 // creates a table in $storeLocation/origin
 
 sql(s"""
 |CREATE EXTERNAL TABLE source
-|STORED BY 'carbondata'
+|STORED AS carbondata
 |LOCATION '$storeLocation/origin'
 """.stripMargin)
 checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin"))
@@ -670,7 +760,7 @@ checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin
 Our SDK module currently support writing data in this format.</p>
 <p><strong>Example:</strong></p>
 <pre><code>sql(
-s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbondata' LOCATION
+s"""CREATE EXTERNAL TABLE sdkOutputTable STORED AS carbondata LOCATION
 |'$writerPath' """.stripMargin)
 </code></pre>
 <p>Here writer path will have carbondata and index files.
@@ -839,14 +929,14 @@ This can be achieved by using the alter table set command.</p>
 You can see the column comment of an existing table using describe formatted command.</p>
 <pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type [COMMENT col_comment], ...)]
   [COMMENT table_comment]
-STORED BY 'carbondata'
+STORED AS carbondata
 [TBLPROPERTIES (property_name=property_value, ...)]
 </code></pre>
 <p>Example:</p>
 <pre><code>CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
                               productNumber Int COMMENT 'unique serial number for product')
 COMMENT ?This is table comment?
- STORED BY 'carbondata'
+ STORED AS carbondata
  TBLPROPERTIES ('DICTIONARY_INCLUDE'='productNumber')
 </code></pre>
 <p>You can also SET and UNSET table comment using ALTER command.</p>
@@ -880,7 +970,7 @@ COMMENT ?This is table comment?
                               saleQuantity INT,
                               revenue INT)
 PARTITIONED BY (productCategory STRING, productBatch STRING)
-STORED BY 'carbondata'
+STORED AS carbondata
 </code></pre>
 <p>NOTE: Hive partition is not supported on complex datatype columns.</p>
 <h4>
@@ -918,7 +1008,7 @@ WHERE au.country = 'US';
 <pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
                   [(col_name data_type , ...)]
 PARTITIONED BY (partition_col_name data_type)
-STORED BY 'carbondata'
+STORED AS carbondata
 [TBLPROPERTIES ('PARTITION_TYPE'='HASH',
                 'NUM_PARTITIONS'='N' ...)]
 </code></pre>
@@ -931,7 +1021,7 @@ STORED BY 'carbondata'
     col_D DECIMAL(10,2),
     col_F TIMESTAMP
 ) PARTITIONED BY (col_E LONG)
-STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='9')
+STORED AS carbondata TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='9')
 </code></pre>
 <h3>
 <a id="create-range-partition-table" class="anchor" href="#create-range-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Range Partition Table</h3>
@@ -939,7 +1029,7 @@ STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='9
 <pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
                   [(col_name data_type , ...)]
 PARTITIONED BY (partition_col_name data_type)
-STORED BY 'carbondata'
+STORED AS carbondata
 [TBLPROPERTIES ('PARTITION_TYPE'='RANGE',
                 'RANGE_INFO'='2014-01-01, 2015-01-01, 2016-01-01, ...')]
 </code></pre>
@@ -966,7 +1056,7 @@ STORED BY 'carbondata'
 <pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
                   [(col_name data_type , ...)]
 PARTITIONED BY (partition_col_name data_type)
-STORED BY 'carbondata'
+STORED AS carbondata
 [TBLPROPERTIES ('PARTITION_TYPE'='LIST',
                 'LIST_INFO'='A, B, C, ...')]
 </code></pre>
@@ -979,7 +1069,7 @@ STORED BY 'carbondata'
     col_E LONG,
     col_F TIMESTAMP
  ) PARTITIONED BY (col_A STRING)
- STORED BY 'carbondata'
+ STORED AS carbondata
  TBLPROPERTIES('PARTITION_TYPE'='LIST',
  'LIST_INFO'='aaaa, bbbb, (cccc, dddd), eeee')
 </code></pre>
@@ -1027,7 +1117,7 @@ columns to be used for bucketing and the number of buckets. For the selection of
 of columns is used.</p>
 <pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
                   [(col_name data_type, ...)]
-STORED BY 'carbondata'
+STORED AS carbondata
 TBLPROPERTIES('BUCKETNUMBER'='noOfBuckets',
 'BUCKETCOLUMNS'='columnname')
 </code></pre>
@@ -1038,15 +1128,15 @@ TBLPROPERTIES('BUCKETNUMBER'='noOfBuckets',
 </ul>
 <p>Example:</p>
 <pre><code>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'
+  productNumber INT,
+  saleQuantity INT,
+  productName STRING,
+  storeCity STRING,
+  storeProvince STRING,
+  productCategory STRING,
+  productBatch STRING,
+  revenue INT)
+STORED AS carbondata
 TBLPROPERTIES ('BUCKETNUMBER'='4', 'BUCKETCOLUMNS'='productName')
 </code></pre>
 <script>
@@ -1059,8 +1149,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -1075,4 +1164,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/ddl-operation-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/ddl-operation-on-carbondata.html b/content/ddl-operation-on-carbondata.html
deleted file mode 100644
index 807adf3..0000000
--- a/content/ddl-operation-on-carbondata.html
+++ /dev/null
@@ -1,748 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
-                                   target="_blank">Apache CarbonData 1.2.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
-                                   target="_blank">Apache CarbonData 1.1.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
-                                   target="_blank">Apache CarbonData 1.1.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
-                                   target="_blank">Apache CarbonData 1.0.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
-                                   target="_blank">Apache CarbonData 0.2.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
-                                   target="_blank">Apache CarbonData 0.1.1</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
-                                   target="_blank">Apache CarbonData 0.1.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search" target="_blank">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input"  placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="row">
-                                <div class="col-sm-12  col-md-12">
-                                    <div>
-<h1>
-<a id="ddl-operations-on-carbondata" class="anchor" href="#ddl-operations-on-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DDL Operations on CarbonData</h1>
-<p>This tutorial guides you through the data definition language support provided by CarbonData.</p>
-<h2>
-<a id="overview" class="anchor" href="#overview" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Overview</h2>
-<p>The following DDL operations are supported in CarbonData :</p>
-<ul>
-<li><a href="#create-table">CREATE TABLE</a></li>
-<li><a href="#show-table">SHOW TABLE</a></li>
-<li>
-<a href="#alter-table">ALTER TABLE</a>
-<ul>
-<li><a href="#rename-table">RENAME TABLE</a></li>
-<li><a href="#add-column">ADD COLUMN</a></li>
-<li><a href="#drop-columns">DROP COLUMNS</a></li>
-<li><a href="#change-data-type">CHANGE DATA TYPE</a></li>
-</ul>
-</li>
-<li><a href="#drop-table">DROP TABLE</a></li>
-<li><a href="#compaction">COMPACTION</a></li>
-<li><a href="#bucketing">BUCKETING</a></li>
-</ul>
-<h2>
-<a id="create-table" class="anchor" href="#create-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE</h2>
-<p>This command can be used to create a CarbonData table by specifying the list of fields along with the table properties.</p>
-<pre><code>   CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
-                    [(col_name data_type , ...)]
-   STORED BY 'carbondata'
-   [TBLPROPERTIES (property_name=property_value, ...)]
-   // All Carbon's additional table options will go into properties
-</code></pre>
-<h3>
-<a id="parameter-description" class="anchor" href="#parameter-description" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Optional</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>db_name</td>
-<td>Name of the database. Database name should consist of alphanumeric characters and underscore(_) special character.</td>
-<td>YES</td>
-</tr>
-<tr>
-<td>field_list</td>
-<td>Comma separated List of fields with data type. The field names should consist of alphanumeric characters and underscore(_) special character.</td>
-<td>NO</td>
-</tr>
-<tr>
-<td>table_name</td>
-<td>The name of the table in Database. Table name should consist of alphanumeric characters and underscore(_) special character.</td>
-<td>NO</td>
-</tr>
-<tr>
-<td>STORED BY</td>
-<td>"org.apache.carbondata.format", identifies and creates a CarbonData table.</td>
-<td>NO</td>
-</tr>
-<tr>
-<td>TBLPROPERTIES</td>
-<td>List of CarbonData table properties.</td>
-<td>YES</td>
-</tr>
-</tbody>
-</table>
-<h3>
-<a id="usage-guidelines" class="anchor" href="#usage-guidelines" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
-<p>Following are the guidelines for using table properties.</p>
-<ul>
-<li>
-<p><strong>Dictionary Encoding Configuration</strong></p>
-<p>Dictionary encoding is enabled by default for all String columns, and disabled for non-String columns. You can include and exclude columns for dictionary encoding.</p>
-</li>
-</ul>
-<pre><code>       TBLPROPERTIES ('DICTIONARY_EXCLUDE'='column1, column2')
-       TBLPROPERTIES ('DICTIONARY_INCLUDE'='column1, column2')
-</code></pre>
-<p>Here, DICTIONARY_EXCLUDE will exclude dictionary creation. This is applicable for high-cardinality columns and is an optional parameter. DICTIONARY_INCLUDE will generate dictionary for the columns specified in the list.</p>
-<ul>
-<li>
-<p><strong>Table Block Size Configuration</strong></p>
-<p>The block size of table files can be defined using the property TABLE_BLOCKSIZE. It accepts only integer values. The default value is 1024 MB and supports a range of 1 MB to 2048 MB.
-If you do not specify this value in the DDL command, default value is used.</p>
-</li>
-</ul>
-<pre><code>       TBLPROPERTIES ('TABLE_BLOCKSIZE'='512')
-</code></pre>
-<p>Here 512 MB means the block size of this table is 512 MB, you can also set it as 512M or 512.</p>
-<ul>
-<li>
-<p><strong>Inverted Index Configuration</strong></p>
-<p>Inverted index is very useful to improve compression ratio and query speed, especially for those low-cardinality columns which are in reward position.
-By default inverted index is enabled. The user can disable the inverted index creation for some columns.</p>
-</li>
-</ul>
-<pre><code>       TBLPROPERTIES ('NO_INVERTED_INDEX'='column1, column3')
-</code></pre>
-<p>No inverted index shall be generated for the columns specified in NO_INVERTED_INDEX. This property is applicable on columns with high-cardinality and is an optional parameter.</p>
-<p>NOTE:</p>
-<ul>
-<li>
-<p>By default all columns other than numeric datatype are treated as dimensions and all columns of numeric datatype are treated as measures.</p>
-</li>
-<li>
-<p>All dimensions except complex datatype columns are part of multi dimensional key(MDK). This behavior can be overridden by using TBLPROPERTIES. If the user wants to keep any column (except columns of complex datatype) in multi dimensional key then he can keep the columns either in DICTIONARY_EXCLUDE or DICTIONARY_INCLUDE.</p>
-</li>
-<li>
-<p><strong>Sort Columns Configuration</strong></p>
-<p>"SORT_COLUMN" property is for users to specify which columns belong to the MDK index. If user don't specify "SORT_COLUMN" property, by default MDK index be built by using all dimension columns except complex datatype column.</p>
-</li>
-</ul>
-<pre><code>       TBLPROPERTIES ('SORT_COLUMNS'='column1, column3')
-</code></pre>
-<h3>
-<a id="example" class="anchor" href="#example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
-<pre><code>    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_EXCLUDE'='storeCity',
-                     'DICTIONARY_INCLUDE'='productNumber',
-                     'NO_INVERTED_INDEX'='productBatch',
-                     'SORT_COLUMNS'='productName,storeCity')
-</code></pre>
-<ul>
-<li><strong>SORT_COLUMNS</strong></li>
-</ul>
-<pre><code>This table property specifies the order of the sort column.
-</code></pre>
-<pre><code>    TBLPROPERTIES('SORT_COLUMNS'='column1, column3')
-</code></pre>
-<p>NOTE:</p>
-<ul>
-<li>
-<p>If this property is not specified, then by default SORT_COLUMNS consist of all dimension (exclude Complex Column).</p>
-</li>
-<li>
-<p>If this property is specified but with empty argument, then the table will be loaded without sort. For example, ('SORT_COLUMNS'='')</p>
-</li>
-</ul>
-<h2>
-<a id="show-table" class="anchor" href="#show-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW TABLE</h2>
-<p>This command can be used to list all the tables in current database or all the tables of a specific database.</p>
-<pre><code>  SHOW TABLES [IN db_Name];
-</code></pre>
-<h3>
-<a id="parameter-description-1" class="anchor" href="#parameter-description-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Optional</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>IN db_Name</td>
-<td>Name of the database. Required only if tables of this specific database are to be listed.</td>
-<td>YES</td>
-</tr>
-</tbody>
-</table>
-<h3>
-<a id="example-1" class="anchor" href="#example-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
-<pre><code>  SHOW TABLES IN ProductSchema;
-</code></pre>
-<h2>
-<a id="alter-table" class="anchor" href="#alter-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>ALTER TABLE</h2>
-<p>The following section shall discuss the commands to modify the physical or logical state of the existing table(s).</p>
-<h3>
-<a id="rename-table" class="anchor" href="#rename-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><strong>RENAME TABLE</strong>
-</h3>
-<p>This command is used to rename the existing table.</p>
-<pre><code>    ALTER TABLE [db_name.]table_name RENAME TO new_table_name;
-</code></pre>
-<h4>
-<a id="parameter-description-2" class="anchor" href="#parameter-description-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h4>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Optional</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>db_Name</td>
-<td>Name of the database. If this parameter is left unspecified, the current database is selected.</td>
-<td>YES</td>
-</tr>
-<tr>
-<td>table_name</td>
-<td>Name of the existing table.</td>
-<td>NO</td>
-</tr>
-<tr>
-<td>new_table_name</td>
-<td>New table name for the existing table.</td>
-<td>NO</td>
-</tr>
-</tbody>
-</table>
-<h4>
-<a id="usage-guidelines-1" class="anchor" href="#usage-guidelines-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h4>
-<ul>
-<li>
-<p>Queries that require the formation of path using the table name for reading carbon store files, running in parallel with Rename command might fail during the renaming operation.</p>
-</li>
-<li>
-<p>Renaming of Secondary index table(s) is not permitted.</p>
-</li>
-</ul>
-<h4>
-<a id="examples" class="anchor" href="#examples" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples:</h4>
-<pre><code>    ALTER TABLE carbon RENAME TO carbondata;
-</code></pre>
-<pre><code>    ALTER TABLE test_db.carbon RENAME TO test_db.carbondata;
-</code></pre>
-<h3>
-<a id="add-column" class="anchor" href="#add-column" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><strong>ADD COLUMN</strong>
-</h3>
-<p>This command is used to add a new column to the existing table.</p>
-<pre><code>    ALTER TABLE [db_name.]table_name ADD COLUMNS (col_name data_type,...)
-    TBLPROPERTIES('DICTIONARY_INCLUDE'='col_name,...',
-    'DICTIONARY_EXCLUDE'='col_name,...',
-    'DEFAULT.VALUE.COLUMN_NAME'='default_value');
-</code></pre>
-<h4>
-<a id="parameter-description-3" class="anchor" href="#parameter-description-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h4>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Optional</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>db_Name</td>
-<td>Name of the database. If this parameter is left unspecified, the current database is selected.</td>
-<td>YES</td>
-</tr>
-<tr>
-<td>table_name</td>
-<td>Name of the existing table.</td>
-<td>NO</td>
-</tr>
-<tr>
-<td>col_name data_type</td>
-<td>Name of comma-separated column with data type. Column names contain letters, digits, and underscores (_).</td>
-<td>NO</td>
-</tr>
-</tbody>
-</table>
-<p>NOTE: Do not name the column after name, tupleId, PositionId, and PositionReference when creating Carbon tables because they are used internally by UPDATE, DELETE, and secondary index.</p>
-<h4>
-<a id="usage-guidelines-2" class="anchor" href="#usage-guidelines-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h4>
-<ul>
-<li>
-<p>Apart from DICTIONARY_INCLUDE, DICTIONARY_EXCLUDE and default_value no other property will be read. If any other property name is specified, error will not be thrown, it will be ignored.</p>
-</li>
-<li>
-<p>If default value is not specified, then NULL will be considered as the default value for the column.</p>
-</li>
-<li>
-<p>For addition of column, if DICTIONARY_INCLUDE and DICTIONARY_EXCLUDE are not specified, then the decision will be taken based on data type of the column.</p>
-</li>
-</ul>
-<h4>
-<a id="examples-1" class="anchor" href="#examples-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples:</h4>
-<pre><code>    ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING);
-</code></pre>
-<pre><code>    ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING)
-    TBLPROPERTIES('DICTIONARY_EXCLUDE'='b1');
-</code></pre>
-<pre><code>    ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING)
-    TBLPROPERTIES('DICTIONARY_INCLUDE'='a1');
-</code></pre>
-<pre><code>    ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING)
-    TBLPROPERTIES('DEFAULT.VALUE.a1'='10');
-</code></pre>
-<h3>
-<a id="drop-columns" class="anchor" href="#drop-columns" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><strong>DROP COLUMNS</strong>
-</h3>
-<p>This command is used to delete a existing column or multiple columns in a table.</p>
-<pre><code>    ALTER TABLE [db_name.]table_name DROP COLUMNS (col_name, ...);
-</code></pre>
-<h4>
-<a id="parameter-description-4" class="anchor" href="#parameter-description-4" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h4>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Optional</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>db_Name</td>
-<td>Name of the database. If this parameter is left unspecified, the current database is selected.</td>
-<td>YES</td>
-</tr>
-<tr>
-<td>table_name</td>
-<td>Name of the existing table.</td>
-<td>NO</td>
-</tr>
-<tr>
-<td>col_name</td>
-<td>Name of comma-separated column with data type. Column names contain letters, digits, and underscores (_)</td>
-<td>NO</td>
-</tr>
-</tbody>
-</table>
-<h4>
-<a id="usage-guidelines-3" class="anchor" href="#usage-guidelines-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h4>
-<ul>
-<li>
-<p>Deleting a column will also clear the dictionary files, provided the column is of type dictionary.</p>
-</li>
-<li>
-<p>For delete column operation, there should be at least one key column that exists in the schema after deletion else error message will be displayed and the operation shall fail.</p>
-</li>
-</ul>
-<h4>
-<a id="examples-2" class="anchor" href="#examples-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples:</h4>
-<p>If the table contains 4 columns namely a1, b1, c1, and d1.</p>
-<ul>
-<li><strong>To delete a single column:</strong></li>
-</ul>
-<pre><code>   ALTER TABLE carbon DROP COLUMNS (b1);
-</code></pre>
-<pre><code>    ALTER TABLE test_db.carbon DROP COLUMNS (b1);
-</code></pre>
-<ul>
-<li><strong>To delete multiple columns:</strong></li>
-</ul>
-<pre><code>   ALTER TABLE carbon DROP COLUMNS (c1,d1);
-</code></pre>
-<h3>
-<a id="change-data-type" class="anchor" href="#change-data-type" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><strong>CHANGE DATA TYPE</strong>
-</h3>
-<p>This command is used to change the data type from INT to BIGINT or decimal precision from lower to higher.</p>
-<pre><code>    ALTER TABLE [db_name.]table_name
-    CHANGE col_name col_name changed_column_type;
-</code></pre>
-<h4>
-<a id="parameter-description-5" class="anchor" href="#parameter-description-5" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h4>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Optional</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>db_Name</td>
-<td>Name of the database. If this parameter is left unspecified, the current database is selected.</td>
-<td>YES</td>
-</tr>
-<tr>
-<td>table_name</td>
-<td>Name of the existing table.</td>
-<td>NO</td>
-</tr>
-<tr>
-<td>col_name</td>
-<td>Name of comma-separated column with data type. Column names contain letters, digits, and underscores (_).</td>
-<td>NO</td>
-</tr>
-<tr>
-<td>changed_column_type</td>
-<td>The change in the data type.</td>
-<td>NO</td>
-</tr>
-</tbody>
-</table>
-<h4>
-<a id="usage-guidelines-4" class="anchor" href="#usage-guidelines-4" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h4>
-<ul>
-<li>Change of decimal data type from lower precision to higher precision will only be supported for cases where there is no data loss.</li>
-</ul>
-<h4>
-<a id="valid-scenarios" class="anchor" href="#valid-scenarios" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Valid Scenarios</h4>
-<ul>
-<li>
-<p>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.</p>
-</li>
-<li>
-<p>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.</p>
-</li>
-<li>
-<p>Note :The allowed range is 38,38 (precision, scale) and is a valid upper case scenario which is not resulting in data loss.</p>
-</li>
-</ul>
-<h4>
-<a id="examples-3" class="anchor" href="#examples-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples:</h4>
-<ul>
-<li><strong>Changing data type of column a1 from INT to BIGINT</strong></li>
-</ul>
-<pre><code>   ALTER TABLE test_db.carbon CHANGE a1 a1 BIGINT;
-</code></pre>
-<ul>
-<li><strong>Changing decimal precision of column a1 from 10 to 18.</strong></li>
-</ul>
-<pre><code>   ALTER TABLE test_db.carbon CHANGE a1 a1 DECIMAL(18,2);
-</code></pre>
-<h2>
-<a id="drop-table" class="anchor" href="#drop-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DROP TABLE</h2>
-<p>This command is used to delete an existing table.</p>
-<pre><code>  DROP TABLE [IF EXISTS] [db_name.]table_name;
-</code></pre>
-<h3>
-<a id="parameter-description-6" class="anchor" href="#parameter-description-6" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Optional</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>db_Name</td>
-<td>Name of the database. If not specified, current database will be selected.</td>
-<td>YES</td>
-</tr>
-<tr>
-<td>table_name</td>
-<td>Name of the table to be deleted.</td>
-<td>NO</td>
-</tr>
-</tbody>
-</table>
-<h3>
-<a id="example-2" class="anchor" href="#example-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
-<pre><code>  DROP TABLE IF EXISTS productSchema.productSalesTable;
-</code></pre>
-<h2>
-<a id="compaction" class="anchor" href="#compaction" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COMPACTION</h2>
-<p>This command merges the specified number of segments into one segment. This enhances the query performance of the table.</p>
-<pre><code>  ALTER TABLE [db_name.]table_name COMPACT 'MINOR/MAJOR';
-</code></pre>
-<p>To get details about Compaction refer to <a href="data-management.html">Data Management</a></p>
-<h3>
-<a id="parameter-description-7" class="anchor" href="#parameter-description-7" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Optional</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>db_name</td>
-<td>Database name, if it is not specified then it uses current database.</td>
-<td>YES</td>
-</tr>
-<tr>
-<td>table_name</td>
-<td>The name of the table in provided database.</td>
-<td>NO</td>
-</tr>
-</tbody>
-</table>
-<h3>
-<a id="syntax" class="anchor" href="#syntax" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Syntax</h3>
-<ul>
-<li><strong>Minor Compaction</strong></li>
-</ul>
-<pre><code>ALTER TABLE table_name COMPACT 'MINOR';
-</code></pre>
-<ul>
-<li><strong>Major Compaction</strong></li>
-</ul>
-<pre><code>ALTER TABLE table_name COMPACT 'MAJOR';
-</code></pre>
-<h2>
-<a id="bucketing" class="anchor" href="#bucketing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>BUCKETING</h2>
-<p>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, a 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.</p>
-<pre><code>   CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
-                    [(col_name data_type, ...)]
-   STORED BY 'carbondata'
-   TBLPROPERTIES('BUCKETNUMBER'='noOfBuckets',
-   'BUCKETCOLUMNS'='columnname')
-</code></pre>
-<h3>
-<a id="parameter-description-8" class="anchor" href="#parameter-description-8" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Optional</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>BUCKETNUMBER</td>
-<td>Specifies the number of Buckets to be created.</td>
-<td>No</td>
-</tr>
-<tr>
-<td>BUCKETCOLUMNS</td>
-<td>Specify the columns to be considered for Bucketing</td>
-<td>No</td>
-</tr>
-</tbody>
-</table>
-<h3>
-<a id="usage-guidelines-5" class="anchor" href="#usage-guidelines-5" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
-<ul>
-<li>
-<p>The feature is supported for Spark 1.6.2 onwards, but the performance optimization is evident from Spark 2.1 onwards.</p>
-</li>
-<li>
-<p>Bucketing can not be performed for columns of Complex Data Types.</p>
-</li>
-<li>
-<p>Columns in the BUCKETCOLUMN parameter must be only dimension. The BUCKETCOLUMN parameter can not be a measure or a combination of measures and dimensions.</p>
-</li>
-</ul>
-<h3>
-<a id="example-3" class="anchor" href="#example-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
-<pre><code> 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 ('DICTIONARY_EXCLUDE'='productName',
-                  'DICTIONARY_INCLUDE'='productNumber,saleQuantity',
-                  'NO_INVERTED_INDEX'='productBatch',
-                  'BUCKETNUMBER'='4',
-                  'BUCKETCOLUMNS'='productName')
-</code></pre>
-</div>
-</div>
-</div>
-</div>
-<div class="doc-footer">
-    <a href="#top" class="scroll-top">Top</a>
-</div>
-</div>
-</section>
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/dml-of-carbondata.html
----------------------------------------------------------------------
diff --git a/content/dml-of-carbondata.html b/content/dml-of-carbondata.html
index 2048c9e..ac41f7c 100644
--- a/content/dml-of-carbondata.html
+++ b/content/dml-of-carbondata.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -231,7 +231,101 @@
 INTO TABLE [db_name.]table_name 
 OPTIONS(property_name=property_value, ...)
 </code></pre>
-<p><strong>Supported Properties:</strong> <a href="#delimiter">DELIMITER</a>, <a href="#quotechar">QUOTECHAR</a>, <a href="#commentchar">COMMENTCHAR</a>, <a href="#header">HEADER</a>, <a href="#fileheader">FILEHEADER</a>, <a href="#multiline">MULTILINE</a>, <a href="#escapechar">ESCAPECHAR</a>, <a href="#skip_empty_line">SKIP_EMPTY_LINE</a>, <a href="#complex_delimiter_level_1">COMPLEX_DELIMITER_LEVEL_1</a>, <a href="#complex_delimiter_level_2">COMPLEX_DELIMITER_LEVEL_2</a>, <a href="#all_dictionary_path">ALL_DICTIONARY_PATH</a>, <a href="#columndict">COLUMNDICT</a>, <a href="#dateformat">DATEFORMAT</a>,<a href="#timestampformat"> TIMESTAMPFORMAT</a>, <a href="#sort-column-bounds">SORT_COLUMN_BOUNDS</a>, <a href="#single_pass">SINGLE_PASS</a>, <a href="#bad-records-handling">BAD_RECORDS_LOGGER_ENABLE</a>, <a href="#bad-records-handling">BAD_RECORD_PATH</a>, <a href="#bad-records-handling">BAD_RECORDS_ACTION</a>, <a href="#bad-records-handling">IS_EMPTY_DATA_BAD_RECORD</a>, <a href="
 #global_sort_partitions">GLOBAL_SORT_PARTITIONS</a></p>
+<p><strong>Supported Properties:</strong></p>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td><a href="#delimiter">DELIMITER</a></td>
+<td>Character used to separate the data in the input csv file</td>
+</tr>
+<tr>
+<td><a href="#quotechar">QUOTECHAR</a></td>
+<td>Character used to quote the data in the input csv file</td>
+</tr>
+<tr>
+<td><a href="#commentchar">COMMENTCHAR</a></td>
+<td>Character used to comment the rows in the input csv file.Those rows will be skipped from processing</td>
+</tr>
+<tr>
+<td><a href="#header">HEADER</a></td>
+<td>Whether the input csv files have header row</td>
+</tr>
+<tr>
+<td><a href="#fileheader">FILEHEADER</a></td>
+<td>If header is not present in the input csv, what is the column names to be used for data read from input csv</td>
+</tr>
+<tr>
+<td><a href="#multiline">MULTILINE</a></td>
+<td>Whether a row data can span across multiple lines.</td>
+</tr>
+<tr>
+<td><a href="#escapechar">ESCAPECHAR</a></td>
+<td>Escape character used to excape the data in input csv file.For eg.,\ is a standard escape character</td>
+</tr>
+<tr>
+<td><a href="#skip_empty_line">SKIP_EMPTY_LINE</a></td>
+<td>Whether empty lines in input csv file should be skipped or loaded as null row</td>
+</tr>
+<tr>
+<td><a href="#complex_delimiter_level_1">COMPLEX_DELIMITER_LEVEL_1</a></td>
+<td>Starting delimiter for complex type data in input csv file</td>
+</tr>
+<tr>
+<td><a href="#complex_delimiter_level_2">COMPLEX_DELIMITER_LEVEL_2</a></td>
+<td>Ending delimiter for complex type data in input csv file</td>
+</tr>
+<tr>
+<td><a href="#all_dictionary_path">ALL_DICTIONARY_PATH</a></td>
+<td>Path to read the dictionary data from all columns</td>
+</tr>
+<tr>
+<td><a href="#columndict">COLUMNDICT</a></td>
+<td>Path to read the dictionary data from for particular column</td>
+</tr>
+<tr>
+<td><a href="#dateformat">DATEFORMAT</a></td>
+<td>Format of date in the input csv file</td>
+</tr>
+<tr>
+<td><a href="#timestampformat">TIMESTAMPFORMAT</a></td>
+<td>Format of timestamp in the input csv file</td>
+</tr>
+<tr>
+<td><a href="#sort-column-bounds">SORT_COLUMN_BOUNDS</a></td>
+<td>How to parititon the sort columns to make the evenly distributed</td>
+</tr>
+<tr>
+<td><a href="#single_pass">SINGLE_PASS</a></td>
+<td>When to enable single pass data loading</td>
+</tr>
+<tr>
+<td><a href="#bad-records-handling">BAD_RECORDS_LOGGER_ENABLE</a></td>
+<td>Whether to enable bad records logging</td>
+</tr>
+<tr>
+<td><a href="#bad-records-handling">BAD_RECORD_PATH</a></td>
+<td>Bad records logging path.Useful when bad record logging is enabled</td>
+</tr>
+<tr>
+<td><a href="#bad-records-handling">BAD_RECORDS_ACTION</a></td>
+<td>Behavior of data loading when bad record is found</td>
+</tr>
+<tr>
+<td><a href="#bad-records-handling">IS_EMPTY_DATA_BAD_RECORD</a></td>
+<td>Whether empty data of a column to be considered as bad record or not</td>
+</tr>
+<tr>
+<td><a href="#global_sort_partitions">GLOBAL_SORT_PARTITIONS</a></td>
+<td>Number of partition to use for shuffling of data during sorting</td>
+</tr>
+</tbody>
+</table>
 <p>You can use the following options to load data:</p>
 <ul>
 <li>
@@ -569,8 +663,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -585,4 +678,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file


[31/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/dml-operation-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/dml-operation-on-carbondata.html b/content/dml-operation-on-carbondata.html
deleted file mode 100644
index 655a61c..0000000
--- a/content/dml-operation-on-carbondata.html
+++ /dev/null
@@ -1,716 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
-                                   target="_blank">Apache CarbonData 1.2.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
-                                   target="_blank">Apache CarbonData 1.1.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
-                                   target="_blank">Apache CarbonData 1.1.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
-                                   target="_blank">Apache CarbonData 1.0.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
-                                   target="_blank">Apache CarbonData 0.2.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
-                                   target="_blank">Apache CarbonData 0.1.1</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
-                                   target="_blank">Apache CarbonData 0.1.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search" target="_blank">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input"  placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="row">
-                                <div class="col-sm-12  col-md-12">
-                                    <div>
-<h1>
-<a id="dml-operations-on-carbondata" class="anchor" href="#dml-operations-on-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DML Operations on CarbonData</h1>
-<p>This tutorial guides you through the data manipulation language support provided by CarbonData.</p>
-<h2>
-<a id="overview" class="anchor" href="#overview" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Overview</h2>
-<p>The following DML operations are supported in CarbonData :</p>
-<ul>
-<li><a href="#load-data">LOAD DATA</a></li>
-<li><a href="#insert-data-into-a-carbondata-table">INSERT DATA INTO A CARBONDATA TABLE</a></li>
-<li><a href="#show-segments">SHOW SEGMENTS</a></li>
-<li><a href="#delete-segment-by-id">DELETE SEGMENT BY ID</a></li>
-<li><a href="#delete-segment-by-date">DELETE SEGMENT BY DATE</a></li>
-<li><a href="#update-carbondata-table">UPDATE CARBONDATA TABLE</a></li>
-<li><a href="#delete-records-from-carbondata-table">DELETE RECORDS FROM CARBONDATA TABLE</a></li>
-</ul>
-<h2>
-<a id="load-data" class="anchor" href="#load-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LOAD DATA</h2>
-<p>This command loads the user data in raw format to the CarbonData specific data format store, this allows CarbonData to provide good performance while querying the data.
-Please visit <a href="data-management.html">Data Management</a> for more details on LOAD.</p>
-<h3>
-<a id="syntax" class="anchor" href="#syntax" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Syntax</h3>
-<pre><code>LOAD DATA [LOCAL] INPATH 'folder_path' 
-INTO TABLE [db_name.]table_name 
-OPTIONS(property_name=property_value, ...)
-</code></pre>
-<p>OPTIONS are not mandatory for data loading process. Inside OPTIONS user can provide either of any options like DELIMITER, QUOTECHAR, ESCAPECHAR, MULTILINE as per requirement.</p>
-<p>NOTE: The path shall be canonical path.</p>
-<h3>
-<a id="parameter-description" class="anchor" href="#parameter-description" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Optional</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>folder_path</td>
-<td>Path of raw csv data folder or file.</td>
-<td>NO</td>
-</tr>
-<tr>
-<td>db_name</td>
-<td>Database name, if it is not specified then it uses the current database.</td>
-<td>YES</td>
-</tr>
-<tr>
-<td>table_name</td>
-<td>The name of the table in provided database.</td>
-<td>NO</td>
-</tr>
-<tr>
-<td>OPTIONS</td>
-<td>Extra options provided to Load</td>
-<td>YES</td>
-</tr>
-</tbody>
-</table>
-<h3>
-<a id="usage-guidelines" class="anchor" href="#usage-guidelines" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
-<p>You can use the following options to load data:</p>
-<ul>
-<li>
-<p><strong>DELIMITER:</strong> Delimiters can be provided in the load command.</p>
-<pre><code>OPTIONS('DELIMITER'=',')
-</code></pre>
-</li>
-<li>
-<p><strong>QUOTECHAR:</strong> Quote Characters can be provided in the load command.</p>
-<pre><code>OPTIONS('QUOTECHAR'='"')
-</code></pre>
-</li>
-<li>
-<p><strong>COMMENTCHAR:</strong> Comment Characters can be provided in the load command if user want to comment lines.</p>
-<pre><code>OPTIONS('COMMENTCHAR'='#')
-</code></pre>
-</li>
-<li>
-<p><strong>FILEHEADER:</strong> Headers can be provided in the LOAD DATA command if headers are missing in the source files.</p>
-<pre><code>OPTIONS('FILEHEADER'='column1,column2') 
-</code></pre>
-</li>
-<li>
-<p><strong>MULTILINE:</strong> CSV with new line character in quotes.</p>
-<pre><code>OPTIONS('MULTILINE'='true') 
-</code></pre>
-</li>
-<li>
-<p><strong>ESCAPECHAR:</strong> Escape char can be provided if user want strict validation of escape character on CSV.</p>
-<pre><code>OPTIONS('ESCAPECHAR'='\') 
-</code></pre>
-</li>
-<li>
-<p><strong>COMPLEX_DELIMITER_LEVEL_1:</strong> Split the complex type data column in a row (eg., a$b$c --&gt; Array = {a,b,c}).</p>
-<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_1'='$') 
-</code></pre>
-</li>
-<li>
-<p><strong>COMPLEX_DELIMITER_LEVEL_2:</strong> Split the complex type nested data column in a row. Applies level_1 delimiter &amp; applies level_2 based on complex data type (eg., a:b$c:d --&gt; Array&gt; = {{a,b},{c,d}}).</p>
-<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_2'=':')
-</code></pre>
-</li>
-<li>
-<p><strong>ALL_DICTIONARY_PATH:</strong> All dictionary files path.</p>
-<pre><code>OPTIONS('ALL_DICTIONARY_PATH'='/opt/alldictionary/data.dictionary')
-</code></pre>
-</li>
-<li>
-<p><strong>COLUMNDICT:</strong> Dictionary file path for specified column.</p>
-<pre><code>OPTIONS('COLUMNDICT'='column1:dictionaryFilePath1,
-column2:dictionaryFilePath2')
-</code></pre>
-<p>NOTE: ALL_DICTIONARY_PATH and COLUMNDICT can't be used together.</p>
-</li>
-<li>
-<p><strong>DATEFORMAT:</strong> Date format for specified column.</p>
-<pre><code>OPTIONS('DATEFORMAT'='column1:dateFormat1, column2:dateFormat2')
-</code></pre>
-<p>NOTE: Date formats are specified by date pattern strings. The date pattern letters in CarbonData are same as in JAVA. Refer to <a href="http://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html" target=_blank>SimpleDateFormat</a>.</p>
-</li>
-<li>
-<p><strong>SINGLE_PASS:</strong> 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.</p>
-<p>This option specifies whether to use single pass for loading data or not. By default this option is set to FALSE.</p>
-<pre><code>OPTIONS('SINGLE_PASS'='TRUE')
-</code></pre>
-<p>Note :</p>
-<ul>
-<li>
-<p>If this option is set to TRUE then data loading will take less time.</p>
-</li>
-<li>
-<p>If this option is set to some invalid value other than TRUE or FALSE then it uses the default value.</p>
-</li>
-<li>
-<p>If this option is set to TRUE, then high.cardinality.identify.enable property will be disabled during data load.</p>
-</li>
-</ul>
-<h3>
-<a id="example" class="anchor" href="#example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
-</li>
-</ul>
-<pre><code>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'
-)
-</code></pre>
-<ul>
-<li>
-<p><strong>BAD RECORDS HANDLING:</strong> Methods of handling bad records are as follows:</p>
-<ul>
-<li>
-<p>Load all of the data before dealing with the errors.</p>
-</li>
-<li>
-<p>Clean or delete bad records before loading data or stop the loading when bad records are found.</p>
-</li>
-</ul>
-<pre><code>OPTIONS('BAD_RECORDS_LOGGER_ENABLE'='true', 'BAD_RECORD_PATH'='hdfs://hacluster/tmp/carbon', 'BAD_RECORDS_ACTION'='REDIRECT', 'IS_EMPTY_DATA_BAD_RECORD'='false')
-</code></pre>
-<p>NOTE:</p>
-<ul>
-<li>
-<p>If the REDIRECT option is used, Carbon 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.</p>
-</li>
-<li>
-<p>In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails.</p>
-</li>
-<li>
-<p>The maximum number of characters per column is 100000. If there are more than 100000 characters in a column, data loading will fail.</p>
-</li>
-</ul>
-</li>
-</ul>
-<h3>
-<a id="example-1" class="anchor" href="#example-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
-<pre><code>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');
-</code></pre>
-<p><strong>Bad Records Management Options:</strong></p>
-<table>
-<thead>
-<tr>
-<th>Options</th>
-<th>Default Value</th>
-<th>Description</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>BAD_RECORDS_LOGGER_ENABLE</td>
-<td>false</td>
-<td>Whether to create logs with details about bad records.</td>
-</tr>
-<tr>
-<td>BAD_RECORDS_ACTION</td>
-<td>FAIL</td>
-<td>Following are the four types of action for bad records:  FORCE: Auto-corrects the data by storing the bad records as NULL.  REDIRECT: Bad records are written to the raw CSV instead of being loaded.  IGNORE: Bad records are neither loaded nor written to the raw CSV.  FAIL: Data loading fails if any bad records are found.  NOTE: In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails.</td>
-</tr>
-<tr>
-<td>IS_EMPTY_DATA_BAD_RECORD</td>
-<td>false</td>
-<td>If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa.</td>
-</tr>
-<tr>
-<td>BAD_RECORD_PATH</td>
-<td>-</td>
-<td>Specifies the HDFS path where bad records are stored. By default the value is Null. This path must to be configured by the user if bad record logger is enabled or bad record action redirect.</td>
-</tr>
-</tbody>
-</table>
-<h2>
-<a id="insert-data-into-a-carbondata-table" class="anchor" href="#insert-data-into-a-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>INSERT DATA INTO A CARBONDATA TABLE</h2>
-<p>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.</p>
-<p><strong>NOTE</strong> :  The client node where the INSERT command is executing, must be part of the cluster.</p>
-<h3>
-<a id="syntax-1" class="anchor" href="#syntax-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Syntax</h3>
-<pre><code>INSERT INTO TABLE &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
-[ WHERE { &lt;filter_condition&gt; } ];
-</code></pre>
-<p>You can also omit the <code>table</code> keyword and write your query as:</p>
-<pre><code>INSERT INTO &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
-[ WHERE { &lt;filter_condition&gt; } ];
-</code></pre>
-<h3>
-<a id="parameter-description-1" class="anchor" href="#parameter-description-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>CARBON TABLE</td>
-<td>The name of the Carbon table in which you want to perform the insert operation.</td>
-</tr>
-<tr>
-<td>sourceTableName</td>
-<td>The table from which the records are read and inserted into destination CarbonData table.</td>
-</tr>
-</tbody>
-</table>
-<h3>
-<a id="usage-guidelines-1" class="anchor" href="#usage-guidelines-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
-<p>The following condition must be met for successful insert operation :</p>
-<ul>
-<li>The source table and the CarbonData table must have the same table schema.</li>
-<li>The table must be created.</li>
-<li>Overwrite is not supported for CarbonData table.</li>
-<li>The data type of source and destination table columns should be same, else the data from source table will be treated as bad records and the INSERT command fails.</li>
-<li>INSERT INTO command does not support partial success if bad records are found, it will fail.</li>
-<li>Data cannot be loaded or updated in source table while insert from source table to target table is in progress.</li>
-</ul>
-<p>To enable data load or update during insert operation, configure the following property to true.</p>
-<pre><code>carbon.insert.persist.enable=true
-</code></pre>
-<p>By default the above configuration will be false.</p>
-<p><strong>NOTE</strong>: Enabling this property will reduce the performance.</p>
-<h3>
-<a id="examples" class="anchor" href="#examples" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples</h3>
-<pre><code>INSERT INTO table1 SELECT item1 ,sum(item2 + 1000) as result FROM 
-table2 group by item1;
-</code></pre>
-<pre><code>INSERT INTO table1 SELECT item1, item2, item3 FROM table2 
-where item2='xyz';
-</code></pre>
-<pre><code>INSERT INTO table1 SELECT * FROM table2 
-where exists (select * from table3 
-where table2.item1 = table3.item1);
-</code></pre>
-<p><strong>The Status Success/Failure shall be captured in the driver log.</strong></p>
-<h2>
-<a id="show-segments" class="anchor" href="#show-segments" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW SEGMENTS</h2>
-<p>This command is used to get the segments of CarbonData table.</p>
-<pre><code>SHOW SEGMENTS FOR TABLE [db_name.]table_name 
-LIMIT number_of_segments;
-</code></pre>
-<h3>
-<a id="parameter-description-2" class="anchor" href="#parameter-description-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Optional</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>db_name</td>
-<td>Database name, if it is not specified then it uses the current database.</td>
-<td>YES</td>
-</tr>
-<tr>
-<td>table_name</td>
-<td>The name of the table in provided database.</td>
-<td>NO</td>
-</tr>
-<tr>
-<td>number_of_segments</td>
-<td>Limit the output to this number.</td>
-<td>YES</td>
-</tr>
-</tbody>
-</table>
-<h3>
-<a id="example-2" class="anchor" href="#example-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
-<pre><code>SHOW SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4;
-</code></pre>
-<h2>
-<a id="delete-segment-by-id" class="anchor" href="#delete-segment-by-id" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY ID</h2>
-<p>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.</p>
-<p>The following command will get the segmentID.</p>
-<pre><code>SHOW SEGMENTS FOR Table [db_name.]table_name LIMIT number_of_segments
-</code></pre>
-<p>After you retrieve the segment ID of the segment that you want to delete, execute the following command to delete the selected segment.</p>
-<pre><code>DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.ID IN (segment_id1, segments_id2, ...)
-</code></pre>
-<h3>
-<a id="parameter-description-3" class="anchor" href="#parameter-description-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Optional</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>segment_id</td>
-<td>Segment Id of the load.</td>
-<td>NO</td>
-</tr>
-<tr>
-<td>db_name</td>
-<td>Database name, if it is not specified then it uses the current database.</td>
-<td>YES</td>
-</tr>
-<tr>
-<td>table_name</td>
-<td>The name of the table in provided database.</td>
-<td>NO</td>
-</tr>
-</tbody>
-</table>
-<h3>
-<a id="example-3" class="anchor" href="#example-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
-<pre><code>DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0);
-DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0,5,8);
-</code></pre>
-<p>NOTE: Here 0.1 is compacted segment sequence id.</p>
-<h2>
-<a id="delete-segment-by-date" class="anchor" href="#delete-segment-by-date" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY DATE</h2>
-<p>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.</p>
-<pre><code>DELETE FROM TABLE [db_name.]table_name 
-WHERE SEGMENT.STARTTIME BEFORE DATE_VALUE
-</code></pre>
-<h3>
-<a id="parameter-description-4" class="anchor" href="#parameter-description-4" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Optional</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>DATE_VALUE</td>
-<td>Valid segment load start time value. All the segments before this specified date will be deleted.</td>
-<td>NO</td>
-</tr>
-<tr>
-<td>db_name</td>
-<td>Database name, if it is not specified then it uses the current database.</td>
-<td>YES</td>
-</tr>
-<tr>
-<td>table_name</td>
-<td>The name of the table in provided database.</td>
-<td>NO</td>
-</tr>
-</tbody>
-</table>
-<h3>
-<a id="example-4" class="anchor" href="#example-4" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
-<pre><code> DELETE FROM TABLE CarbonDatabase.CarbonTable 
- WHERE SEGMENT.STARTTIME BEFORE '2017-06-01 12:05:06';  
-</code></pre>
-<h2>
-<a id="update-carbondata-table" class="anchor" href="#update-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Update CarbonData Table</h2>
-<p>This command will allow to update the carbon table based on the column expression and optional filter conditions.</p>
-<h3>
-<a id="syntax-2" class="anchor" href="#syntax-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Syntax</h3>
-<pre><code> UPDATE &lt;table_name&gt;
- SET (column_name1, column_name2, ... column_name n) =
- (column1_expression , column2_expression, ... column n_expression )
- [ WHERE { &lt;filter_condition&gt; } ];
-</code></pre>
-<p>alternatively the following the command can also be used for updating the CarbonData Table :</p>
-<pre><code>UPDATE &lt;table_name&gt;
-SET (column_name1, column_name2) =
-(select sourceColumn1, sourceColumn2 from sourceTable
-[ WHERE { &lt;filter_condition&gt; } ] )
-[ WHERE { &lt;filter_condition&gt; } ];
-</code></pre>
-<h3>
-<a id="parameter-description-5" class="anchor" href="#parameter-description-5" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>table_name</td>
-<td>The name of the Carbon table in which you want to perform the update operation.</td>
-</tr>
-<tr>
-<td>column_name</td>
-<td>The destination columns to be updated.</td>
-</tr>
-<tr>
-<td>sourceColumn</td>
-<td>The source table column values to be updated in destination table.</td>
-</tr>
-<tr>
-<td>sourceTable</td>
-<td>The table from which the records are updated into destination Carbon table.</td>
-</tr>
-</tbody>
-</table>
-<p>NOTE: This functionality is currently not supported in Spark 2.x and will support soon.</p>
-<h3>
-<a id="usage-guidelines-2" class="anchor" href="#usage-guidelines-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
-<p>The following conditions must be met for successful updation :</p>
-<ul>
-<li>The update command fails if multiple input rows in source table are matched with single row in destination table.</li>
-<li>If the source table generates empty records, the update operation will complete successfully without updating the table.</li>
-<li>If a source table row does not correspond to any of the existing rows in a destination table, the update operation will complete successfully without updating the table.</li>
-<li>In sub-query, if the source table and the target table are same, then the update operation fails.</li>
-<li>If the sub-query used in UPDATE statement contains aggregate method or group by query, then the UPDATE operation fails.</li>
-</ul>
-<h3>
-<a id="examples-1" class="anchor" href="#examples-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples</h3>
-<p>Update is not supported for queries that contain aggregate or group by.</p>
-<pre><code> UPDATE t_carbn01 a
- SET (a.item_type_code, a.profit) = ( SELECT b.item_type_cd,
- sum(b.profit) from t_carbn01b b
- WHERE item_type_cd =2 group by item_type_code);
-</code></pre>
-<p>Here the Update Operation fails as the query contains aggregate function sum(b.profit) and group by clause in the sub-query.</p>
-<pre><code>UPDATE carbonTable1 d
-SET(d.column3,d.column5 ) = (SELECT s.c33 ,s.c55
-FROM sourceTable1 s WHERE d.column1 = s.c11)
-WHERE d.column1 = 'china' EXISTS( SELECT * from table3 o where o.c2 &gt; 1);
-</code></pre>
-<pre><code>UPDATE carbonTable1 d SET (c3) = (SELECT s.c33 from sourceTable1 s
-WHERE d.column1 = s.c11)
-WHERE exists( select * from iud.other o where o.c2 &gt; 1);
-</code></pre>
-<pre><code>UPDATE carbonTable1 SET (c2, c5 ) = (c2 + 1, concat(c5 , "y" ));
-</code></pre>
-<pre><code>UPDATE carbonTable1 d SET (c2, c5 ) = (c2 + 1, "xyx")
-WHERE d.column1 = 'india';
-</code></pre>
-<pre><code>UPDATE carbonTable1 d SET (c2, c5 ) = (c2 + 1, "xyx")
-WHERE d.column1 = 'india'
-and EXISTS( SELECT * FROM table3 o WHERE o.column2 &gt; 1);
-</code></pre>
-<p><strong>The Status Success/Failure shall be captured in the driver log and the client.</strong></p>
-<h2>
-<a id="delete-records-from-carbondata-table" class="anchor" href="#delete-records-from-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Delete Records from CarbonData Table</h2>
-<p>This command allows us to delete records from CarbonData table.</p>
-<h3>
-<a id="syntax-3" class="anchor" href="#syntax-3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Syntax</h3>
-<pre><code>DELETE FROM table_name [WHERE expression];
-</code></pre>
-<h3>
-<a id="parameter-description-6" class="anchor" href="#parameter-description-6" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Parameter Description</h3>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>table_name</td>
-<td>The name of the Carbon table in which you want to perform the delete.</td>
-</tr>
-</tbody>
-</table>
-<p>NOTE: This functionality is currently not supported in Spark 2.x and will support soon.</p>
-<h3>
-<a id="examples-2" class="anchor" href="#examples-2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples</h3>
-<pre><code>DELETE FROM columncarbonTable1 d WHERE d.column1  = 'china';
-</code></pre>
-<pre><code>DELETE FROM dest WHERE column1 IN ('china', 'USA');
-</code></pre>
-<pre><code>DELETE FROM columncarbonTable1
-WHERE column1 IN (SELECT column11 FROM sourceTable2);
-</code></pre>
-<pre><code>DELETE FROM columncarbonTable1
-WHERE column1 IN (SELECT column11 FROM sourceTable2 WHERE
-column1 = 'USA');
-</code></pre>
-<pre><code>DELETE FROM columncarbonTable1 WHERE column2 &gt;= 4;
-</code></pre>
-<p><strong>The Status Success/Failure shall be captured in the driver log and the client.</strong></p>
-</div>
-</div>
-</div>
-</div>
-<div class="doc-footer">
-    <a href="#top" class="scroll-top">Top</a>
-</div>
-</div>
-</section>
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/documentation.html
----------------------------------------------------------------------
diff --git a/content/documentation.html b/content/documentation.html
index 634da94..982becf 100644
--- a/content/documentation.html
+++ b/content/documentation.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -215,13 +215,13 @@
 <p>Apache CarbonData is a new big data file format for faster interactive query using advanced columnar storage, index, compression and encoding techniques to improve computing efficiency, which helps in speeding up queries by an order of magnitude faster over PetaBytes of data.</p>
 <h2>
 <a id="getting-started" class="anchor" href="#getting-started" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Getting Started</h2>
-<p><strong>File Format Concepts:</strong> Start with the basics of understanding the <a href="./file-structure-of-carbondata.html#carbondata-file-structure">CarbonData file format</a> and its storage structure.This will help to understand other parts of the documentation, incuding deployment, programming and usage guides.</p>
+<p><strong>File Format Concepts:</strong> Start with the basics of understanding the <a href="./file-structure-of-carbondata.html#carbondata-file-format">CarbonData file format</a> and its <a href="./file-structure-of-carbondata.html">storage structure</a>.This will help to understand other parts of the documentation, including deployment, programming and usage guides.</p>
 <p><strong>Quick Start:</strong> <a href="./quick-start-guide.html#installing-and-configuring-carbondata-to-run-locally-with-spark-shell">Run an example program</a> on your local machine or <a href="https://github.com/apache/carbondata/tree/master/examples/spark2/src/main/scala/org/apache/carbondata/examples" target=_blank>study some examples</a>.</p>
 <p><strong>CarbonData SQL Language Reference:</strong> CarbonData extends the Spark SQL language and adds several <a href="./ddl-of-carbondata.html">DDL</a> and <a href="./dml-of-carbondata.html">DML</a> statements to support operations on it.Refer to the <a href="./language-manual.html">Reference Manual</a> to understand the supported features and functions.</p>
 <p><strong>Programming Guides:</strong> You can read our guides about <a href="./sdk-guide.html">APIs supported</a> to learn how to integrate CarbonData with your applications.</p>
 <h2>
-<a id="deployment" class="anchor" href="#deployment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deployment</h2>
-<p>CarbonData can be integrated with popular Execution engines like <a href="./quick-start-guide.html#spark">Spark</a> and <a href="./quick-start-guide.html#presto">Presto</a>.Refer to the <a href="./quick-start-guide.html##deployment-modes">Installation and Configuration</a> section to understand all modes of Integrating CarbonData.</p>
+<a id="integration" class="anchor" href="#integration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Integration</h2>
+<p>CarbonData can be integrated with popular Execution engines like <a href="./quick-start-guide.html#spark">Spark</a> and <a href="./quick-start-guide.html#presto">Presto</a>.Refer to the <a href="./quick-start-guide.html#integration">Installation and Configuration</a> section to understand all modes of Integrating CarbonData.</p>
 <h2>
 <a id="contributing-to-carbondata" class="anchor" href="#contributing-to-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contributing to CarbonData</h2>
 <p>The Apache CarbonData community welcomes all kinds of contributions from anyone with a passion for
@@ -234,10 +234,6 @@ faster data format.Contributing to CarbonData doesn?t just mean writing code. He
 <p><strong>Blogs:</strong> Blogs by external users can be found <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=67635497" rel="nofollow">here</a>.</p>
 <p><strong>Performance reports:</strong> TPC-H performance reports can be found <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Performance+-+TPCH+Report+of+CarbonData+%281.2+version%29+and+Parquet+on+Spark+Execution+Engine" rel="nofollow">here</a>.</p>
 <p><strong>Trainings:</strong> Training records on design and code flows can be found <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/CarbonData+Training+Materials" rel="nofollow">here</a>.</p>
-<script>
-// Show selected style on nav item
-$(function() { $('.b-nav__intro').addClass('selected'); });
-</script>
 </div>
 </div>
 </div>
@@ -253,4 +249,4 @@ $(function() { $('.b-nav__intro').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/errorpage.html
----------------------------------------------------------------------
diff --git a/content/errorpage.html b/content/errorpage.html
index 35cc01a..090dce5 100644
--- a/content/errorpage.html
+++ b/content/errorpage.html
@@ -89,4 +89,4 @@
 
 
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/faq.html
----------------------------------------------------------------------
diff --git a/content/faq.html b/content/faq.html
index 52112b7..c37284f 100644
--- a/content/faq.html
+++ b/content/faq.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -222,19 +222,19 @@
 <li><a href="#how-to-resolve-abstract-method-error">How to resolve Abstract Method Error?</a></li>
 <li><a href="#how-carbon-will-behave-when-execute-insert-operation-in-abnormal-scenarios">How Carbon will behave when execute insert operation in abnormal scenarios?</a></li>
 <li><a href="#why-aggregate-query-is-not-fetching-data-from-aggregate-table">Why aggregate query is not fetching data from aggregate table?</a></li>
-<li><a href="#Why-all-executors-are-showing-success-in-Spark-UI-even-after-Dataload-command-failed-at-driver-side">Why all executors are showing success in Spark UI even after Dataload command failed at Driver side?</a></li>
-<li><a href="#Why-different-time-zone-result-for-select-query-output-when-query-SDK-writer-output">Why different time zone result for select query output when query SDK writer output?</a></li>
+<li><a href="#why-all-executors-are-showing-success-in-spark-ui-even-after-dataload-command-failed-at-driver-side">Why all executors are showing success in Spark UI even after Dataload command failed at Driver side?</a></li>
+<li><a href="#why-different-time-zone-result-for-select-query-output-when-query-sdk-writer-output">Why different time zone result for select query output when query SDK writer output?</a></li>
 </ul>
 <h1>
 <a id="troubleshooting" class="anchor" href="#troubleshooting" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>TroubleShooting</h1>
 <ul>
-<li><a href="#Getting-tablestatus.lock-issues-When-loading-data">Getting tablestatus.lock issues When loading data</a></li>
+<li><a href="#getting-tablestatuslock-issues-when-loading-data">Getting tablestatus.lock issues When loading data</a></li>
 <li><a href="#failed-to-load-thrift-libraries">Failed to load thrift libraries</a></li>
 <li><a href="#failed-to-launch-the-spark-shell">Failed to launch the Spark Shell</a></li>
 <li><a href="#failed-to-execute-load-query-on-cluster">Failed to execute load query on cluster</a></li>
 <li><a href="#failed-to-execute-insert-query-on-cluster">Failed to execute insert query on cluster</a></li>
 <li><a href="#failed-to-connect-to-hiveuser-with-thrift">Failed to connect to hiveuser with thrift</a></li>
-<li><a href="#failed-to-read-the-metastore-db-during-table">Failed to read the metastore db during table</a></li>
+<li><a href="#failed-to-read-the-metastore-db-during-table-creation">Failed to read the metastore db during table creation</a></li>
 <li><a href="#failed-to-load-data-on-the-cluster">Failed to load data on the cluster</a></li>
 <li><a href="#failed-to-insert-data-on-the-cluster">Failed to insert data on the cluster</a></li>
 <li><a href="#failed-to-execute-concurrent-operations-on-table-by-multiple-workers">Failed to execute Concurrent Operations(Load,Insert,Update) on table by multiple workers</a></li>
@@ -292,7 +292,7 @@ The property carbon.lock.type configuration specifies the type of lock to be acq
 <p>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 <code>spark version</code> while using Maven to build project.</p>
 <h2>
 <a id="how-carbon-will-behave-when-execute-insert-operation-in-abnormal-scenarios" class="anchor" href="#how-carbon-will-behave-when-execute-insert-operation-in-abnormal-scenarios" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>How Carbon will behave when execute insert operation in abnormal scenarios?</h2>
-<p>Carbon support insert operation, you can refer to the syntax mentioned in <a href="dml-operation-on-carbondata.html">DML Operations on CarbonData</a>.
+<p>Carbon support insert operation, you can refer to the syntax mentioned in <a href="./dml-of-carbondata.html">DML Operations on CarbonData</a>.
 First, create a source table in spark-sql and load data into this created table.</p>
 <pre><code>CREATE TABLE source_table(
 id String,
@@ -312,7 +312,7 @@ id  name    city
 id String,
 city String,
 name String)
-STORED BY 'carbondata';
+STORED AS carbondata;
 </code></pre>
 <pre><code>INSERT INTO TABLE carbon_table SELECT * FROM source_table;
 </code></pre>
@@ -341,7 +341,7 @@ id  city    name
 When SubQuery predicate is present in the query.</li>
 </ul>
 <p>Example:</p>
-<pre><code>create table gdp21(cntry smallint, gdp double, y_year date) stored by 'carbondata';
+<pre><code>create table gdp21(cntry smallint, gdp double, y_year date) stored as carbondata;
 create datamap ag1 on table gdp21 using 'preaggregate' as select cntry, sum(gdp) from gdp21 group by cntry;
 select ctry from pop1 where ctry in (select cntry from gdp21 group by cntry);
 </code></pre>
@@ -351,7 +351,7 @@ select ctry from pop1 where ctry in (select cntry from gdp21 group by cntry);
 When aggregate function along with 'in' filter.</li>
 </ul>
 <p>Example:</p>
-<pre><code>create table gdp21(cntry smallint, gdp double, y_year date) stored by 'carbondata';
+<pre><code>create table gdp21(cntry smallint, gdp double, y_year date) stored as carbondata;
 create datamap ag1 on table gdp21 using 'preaggregate' as select cntry, sum(gdp) from gdp21 group by cntry;
 select cntry, sum(gdp) from gdp21 where cntry in (select ctry from pop1) group by cntry;
 </code></pre>
@@ -361,7 +361,7 @@ select cntry, sum(gdp) from gdp21 where cntry in (select ctry from pop1) group b
 When aggregate function having 'join' with equal filter.</li>
 </ul>
 <p>Example:</p>
-<pre><code>create table gdp21(cntry smallint, gdp double, y_year date) stored by 'carbondata';
+<pre><code>create table gdp21(cntry smallint, gdp double, y_year date) stored as carbondata;
 create datamap ag1 on table gdp21 using 'preaggregate' as select cntry, sum(gdp) from gdp21 group by cntry;
 select cntry,sum(gdp) from gdp21,pop1 where cntry=ctry group by cntry;
 </code></pre>
@@ -566,8 +566,7 @@ For example, you can use scp to copy this file to all the nodes.</p>
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__faq').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -582,4 +581,4 @@ $(function() { $('.b-nav__faq').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/file-structure-of-carbondata.html
----------------------------------------------------------------------
diff --git a/content/file-structure-of-carbondata.html b/content/file-structure-of-carbondata.html
index b96d622..c14ea6d 100644
--- a/content/file-structure-of-carbondata.html
+++ b/content/file-structure-of-carbondata.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -211,30 +211,138 @@
                                 <div class="col-sm-12  col-md-12">
                                     <div>
 <h1>
-<a id="carbondata-file-structure" class="anchor" href="#carbondata-file-structure" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData File Structure</h1>
+<a id="carbondata-table-structure" class="anchor" href="#carbondata-table-structure" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData table structure</h1>
 <p>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.</p>
 <p>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.</p>
-<h3>
-<a id="understanding-carbondata-file-structure" class="anchor" href="#understanding-carbondata-file-structure" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Understanding CarbonData File Structure</h3>
+<p>This document describes the what a CarbonData table looks like in a HDFS directory, files written and content of each file.</p>
 <ul>
-<li>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.</li>
-</ul>
-<p><a href="../docs/images/carbon_data_file_structure_new.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_file_structure_new.png?raw=true" alt="CarbonData File Structure" style="max-width:100%;"></a></p>
+<li>
+<p><a href="#file-directory-structure">File Directory Structure</a></p>
+</li>
+<li>
+<p><a href="#file-content-details">File Content details</a></p>
 <ul>
-<li>File Header : It contains CarbonData file version number, list of column schema and schema updation timestamp.</li>
-<li>File Footer : it contains Number of rows, segmentinfo ,all blocklets? info and index, you can find the detail from the below diagram.</li>
-<li>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.</li>
-<li>Column Page Group : Data of one column and it is further divided into pages, it is guaranteed to be contiguous in file.</li>
-<li>Page : It has the data of one column and the number of row is fixed to 32000 size.</li>
-</ul>
-<p><a href="../docs/images/carbon_data_format_new.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_format_new.png?raw=true" alt="CarbonData File Format" style="max-width:100%;"></a></p>
-<h3>
-<a id="each-page-contains-three-types-of-data" class="anchor" href="#each-page-contains-three-types-of-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Each page contains three types of data</h3>
+<li><a href="#schema-file-format">Schema file format</a></li>
+<li>
+<a href="#carbondata-file-format">CarbonData file format</a>
 <ul>
-<li>Data Page: Contains the encoded data of a column of columns.</li>
-<li>Row ID Page (optional): Contains the row ID mappings used when the data page is stored as an inverted index.</li>
-<li>RLE Page (optional): Contains additional metadata used when the data page is RLE coded.</li>
+<li>
+<a href="#blocklet-format">Blocklet format</a>
+<ul>
+<li><a href="#v1">V1</a></li>
+<li><a href="#v2">V2</a></li>
+<li><a href="#v3">V3</a></li>
+</ul>
+</li>
+<li><a href="#footer-format">Footer format</a></li>
 </ul>
+</li>
+<li><a href="#carbonindex-file-format">carbonindex file format</a></li>
+<li><a href="#dictionary-file-format">Dictionary file format</a></li>
+<li><a href="#tablestatus-file-format">tablestatus file format</a></li>
+</ul>
+</li>
+</ul>
+<h2>
+<a id="file-directory-structure" class="anchor" href="#file-directory-structure" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>File Directory Structure</h2>
+<p>The CarbonData files are stored in the location specified by the <em><strong>carbon.storelocation</strong></em> configuration (configured in carbon.properties; if not configured, the default is ../carbon.store).</p>
+<p>The file directory structure is as below:</p>
+<p><a href="../docs/images/2-1_1.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-1_1.png?raw=true" alt="File Directory Structure" style="max-width:100%;"></a></p>
+<ol>
+<li>ModifiedTime.htmlt records the timestamp of the metadata with the modification time attribute of the file. When the drop table and create table are used, the modification time of the file is updated.This is common to all databases and hence is kept in parallel to databases</li>
+<li>The <strong>default</strong> is the database name and contains the user tables.default is used when user doesn't specify any database name;else user configured database name will be the directory name. user_table is the table name.</li>
+<li>Metadata directory stores schema files, tablestatus and dictionary files (including .dict, .dictmeta and .sortindex). There are three types of metadata data information files.</li>
+<li>data and index files are stored under directory named <strong>Fact</strong>. The Fact directory has a Part0 partition directory, where 0 is the partition number.</li>
+<li>There is a Segment_0 directory under the Part0 directory, where 0 is the segment number.</li>
+<li>There are two types of files, carbondata and carbonindex, in the Segment_0 directory.</li>
+</ol>
+<h2>
+<a id="file-content-details" class="anchor" href="#file-content-details" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>File Content details</h2>
+<p>When the table is created, the user_table directory is generated, and a schema file is generated in the Metadata directory for recording the table structure.</p>
+<p>When loading data in batches, each batch loading generates a new segment directory. The scheduling tries to control a task processing data loading task on each node. Each task will generate multiple carbondata files and one carbonindex file.</p>
+<p>During  global dictionary generation, if the two-pass scheme is used, before the data is loaded, the corresponding dict, dictmeta and sortindex files are generated for each dictionary-encoded column, and partial dictionary files can be provided by the pre-define dictionary method to reduce the need. A dictionary-encoded column is generated by scanning the full amount of data; a dictionary file of all dictionary code columns can also be provided by the all dictionary method to avoid scanning data. If the single-pass scheme is adopted, the global dictionary code is generated in real time during data loading, and after the data is loaded, the dictionary is solidified into a dictionary file.</p>
+<p>The following sections use the Java object generated by the thrift file describing the carbondata file format to explain the contents of each file one by one (you can also directly read the format defined in the <a href="https://github.com/apache/carbondata/tree/master/format/src/main/thrift" target=_blank>thrift file</a>)</p>
+<h3>
+<a id="schema-file-format" class="anchor" href="#schema-file-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Schema file format</h3>
+<p>The contents of the schema file is as shown below</p>
+<p><a href="../docs/images/2-2_1.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-2_1.png?raw=true" alt="Schema file format" style="max-width:100%;"></a></p>
+<ol>
+<li>TableSchema class
+The TableSchema class does not store the table name, it is infered from the directory name(user_table).
+tableProperties is used to record table-related properties, such as: table_blocksize.</li>
+<li>ColumnSchema class
+Encoders are used to record the encoding used in column storage.
+columnProperties is used to record column related properties.</li>
+<li>BucketingInfo class
+When creating a bucket table, you can specify the number of buckets in the table and the column to splitbuckets.</li>
+<li>DataType class
+Describes the data types supported by CarbonData.</li>
+<li>Encoding class
+Several encodings that may be used in CarbonData files.</li>
+</ol>
+<h3>
+<a id="carbondata-file-format" class="anchor" href="#carbondata-file-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData file format</h3>
+<h4>
+<a id="file-header" class="anchor" href="#file-header" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>File Header</h4>
+<p>It contains CarbonData file version number, list of column schema and schema updation timestamp.</p>
+<p><a href="../docs/images/carbon_data_file_structure_new.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_file_structure_new.png?raw=true" alt="File Header" style="max-width:100%;"></a></p>
+<p>The carbondata file consists of multiple blocklets and footer parts. The blocklet is the dataset inside the carbondata file (the latest V3 format, the default configuration is 64MB), each blocklet contains a ColumnChunk for each column, and a ColumnChunk may contain one or more Column Pages.</p>
+<p>The carbondata file currently supports V1, V2 and V3 versions. The main difference is the change of the blocklet part, which is introduced one by one.</p>
+<h4>
+<a id="blocklet-format" class="anchor" href="#blocklet-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Blocklet format</h4>
+<h5>
+<a id="v1" class="anchor" href="#v1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>V1</h5>
+<p>Blocket consists of all column data pages, RLE pages, and rowID pages. Since the pages in the blocklet are grouped according to the page type, the three pieces of data of each column are distributed and stored in the blocklet, and the offset and length information of all the pages need to be recorded in the footer part.</p>
+<p><a href="../docs/images/2-3_1.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-3_1.png?raw=true" alt="V1" style="max-width:100%;"></a></p>
+<h5>
+<a id="v2" class="anchor" href="#v2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>V2</h5>
+<p>The blocklet consists of ColumnChunk for all columns. The ColumnChunk for a column consists of a ColumnPage, which includes the data chunk header, data page, RLE page, and rowID page. Since ColumnChunk aggregates the three types of Page data of the column together, it can read the column data using fewer readers. Since the header part records the length information of all the pages, the footer part only needs to record the offset and length of the ColumnChunk, and also reduces the amount of footer data.</p>
+<p><a href="../docs/images/2-3_2.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-3_2.png?raw=true" alt="V2" style="max-width:100%;"></a></p>
+<h5>
+<a id="v3" class="anchor" href="#v3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>V3</h5>
+<p>The blocklet is also composed of ColumnChunks of all columns. What is changed is that a ColumnChunk consists of one or more Column Pages, and Column Page adds a new BlockletMinMaxIndex.</p>
+<p>Compared with V2: The blocklet data volume of V2 format defaults to 120,000 lines, and the blocklet data volume of V3 format defaults to 64MB. For the same size data file, the information of the footer part index metadata may be further reduced; meanwhile, the V3 format adds a new page. Level data filtering, and the amount of data per page is only 32,000 lines by default, which is much less than the 120,000 lines of V2 format. The accuracy of data filtering hits further, and more data can be filtered out before decompressing data.</p>
+<p><a href="../docs/images/2-3_3.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-3_3.png?raw=true" alt="V3" style="max-width:100%;"></a></p>
+<h4>
+<a id="footer-format" class="anchor" href="#footer-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Footer format</h4>
+<p>Footer records each carbondata
+All blocklet data distribution information and statistical related metadata information (minmax, startkey/endkey) inside the file.</p>
+<p><a href="../docs/images/2-3_4.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-3_4.png?raw=true" alt="Footer format" style="max-width:100%;"></a></p>
+<ol>
+<li>BlockletInfo3 is used to record the offset and length of all ColumnChunk3.</li>
+<li>SegmentInfo is used to record the number of columns and the cardinality of each column.</li>
+<li>BlockletIndex includes BlockletMinMaxIndex and BlockletBTreeIndex.</li>
+</ol>
+<p>BlockletBTreeIndex is used to record the startkey/endkey of all blocklets in the block. When querying, the startkey/endkey of the query is generated by filtering conditions combined with.htmlkey. With BlocketBtreeIndex, the range of blocklets satisfying the conditions in each block can be delineated.</p>
+<p>BlockletMinMaxIndex is used to record the min/max value of all columns in the blocklet. By using the min/max check on the filter condition, you can skip the block/blocklet that does not satisfy the condition.</p>
+<h3>
+<a id="carbonindex-file-format" class="anchor" href="#carbonindex-file-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>carbonindex file format</h3>
+<p>Extract the BlockletIndex part of the footer part to generate the carbonindex file. Load data in batches, schedule as much as possible to control a node to start a task, each task generates multiple carbondata files and a carbonindex file. The carbonindex file records the index information of all the blocklets in all the carbondata files generated by the task.</p>
+<p>As shown in the figure, the index information corresponding to a block is recorded by a BlockIndex object, including carbondata filename, footer offset and BlockletIndex. The BlockIndex data volume is less than the footer. The file is directly used to build the index on the driver side when querying, without having to skip the footer part of the data volume of multiple data files.</p>
+<p><a href="../docs/images/2-4_1.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-4_1.png?raw=true" alt="carbonindex file format" style="max-width:100%;"></a></p>
+<h3>
+<a id="dictionary-file-format" class="anchor" href="#dictionary-file-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Dictionary file format</h3>
+<p>For each dictionary encoded column, a dictionary file is used to store the dictionary metadata for that column.</p>
+<ol>
+<li>dict file records the distinct value list of a column</li>
+</ol>
+<p>For the first time dataloading, the file is generated using a distinct value list of a column. The value in the file is unordered; the subsequent append is used. In the second step of dataloading (Data Convert Step), the dictionary code column will replace the true value of the data with the dictionary key.</p>
+<p><a href="../docs/images/2-5_1.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-5_1.png?raw=true" alt="Dictionary file format" style="max-width:100%;"></a></p>
+<ol start="2">
+<li>dictmeta records the metadata description of the new distinct value of each dataloading</li>
+</ol>
+<p>The dictionary cache uses this information to incrementally flush the cache.</p>
+<p><a href="../docs/images/2-5_2.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-5_2.png?raw=true" alt="Dictionary Chunk" style="max-width:100%;"></a></p>
+<ol start="3">
+<li>sortindex records the result set of the key code of the dictionary code sorted by value.</li>
+</ol>
+<p>In dataLoading, if there is a new dictionary value, the sortindex file will be regenerated using all the dictionary codes.</p>
+<p>Filtering queries based on dictionary code columns need to convert the value filter filter to the key filter condition. Using the sortindex file, you can quickly construct an ordered value sequence to quickly find the key value corresponding to the value, thus speeding up the conversion process.</p>
+<p><a href="../docs/images/2-5_3.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-5_3.png?raw=true" alt="sortindex file format" style="max-width:100%;"></a></p>
+<h3>
+<a id="tablestatus-file-format" class="anchor" href="#tablestatus-file-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>tablestatus file format</h3>
+<p>Tablestatus records the segment-related information (in gson format) for each load and merge, including load time, load status, segment name, whether it was deleted, and the segment name incorporated. Regenerate the tablestatusfile after each load or merge.</p>
+<p><a href="../docs/images/2-6_1.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/2-6_1.png?raw=true" alt="tablestatus file format" style="max-width:100%;"></a></p>
 <script>
 $(function() {
   // Show selected style on nav item
@@ -245,8 +353,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -261,4 +368,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/how-to-contribute-to-apache-carbondata.html
----------------------------------------------------------------------
diff --git a/content/how-to-contribute-to-apache-carbondata.html b/content/how-to-contribute-to-apache-carbondata.html
index 946013f..122b763 100644
--- a/content/how-to-contribute-to-apache-carbondata.html
+++ b/content/how-to-contribute-to-apache-carbondata.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -352,8 +352,7 @@ $ git push &lt;GitHub_user&gt; --delete &lt;my-branch&gt;
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__contri').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -368,4 +367,4 @@ $(function() { $('.b-nav__contri').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/index.html
----------------------------------------------------------------------
diff --git a/content/index.html b/content/index.html
index e9e6e3b..f059d16 100644
--- a/content/index.html
+++ b/content/index.html
@@ -477,7 +477,7 @@
                         </ol>
                         <p class="title-info">
                             For detailed reference on CarbonData, read the <a
-                                href="mainpage.html">User Guide</a>.
+                                href="documentation.html">User Guide</a>.
                         </p>
                     </div>
 


[07/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/release-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/release-guide.html b/src/main/webapp/release-guide.html
new file mode 100644
index 0000000..fb51368
--- /dev/null
+++ b/src/main/webapp/release-guide.html
@@ -0,0 +1,572 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="apache-carbondata-release-guide" class="anchor" href="#apache-carbondata-release-guide" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Apache CarbonData Release Guide</h1>
+<p>Apache CarbonData periodically declares and publishes releases.</p>
+<p>Each release is executed by a <em>Release Manager</em>, who is selected among the CarbonData committers.
+This document describes the process that the Release Manager follows to perform a release. Any
+changes to this process should be discussed and adopted on the
+<a href="mailto:dev@carbondata.apache.org">dev@ mailing list</a>.</p>
+<p>Please remember that publishing software has legal consequences. This guide complements the
+foundation-wide <a href="http://www.apache.org/dev/release.html" target=_blank rel="nofollow">Product Release Policy</a> and <a href="http://www.apache.org/dev/release-distribution" target=_blank rel="nofollow">Release
+Distribution Policy</a>.</p>
+<h2>
+<a id="decide-to-release" class="anchor" href="#decide-to-release" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Decide to release</h2>
+<p>Deciding to release and selecting a Release Manager is the first step of the release process.
+This is a consensus-based decision of the entire community.</p>
+<p>Anybody can propose a release on the dev@ mailing list, giving a solid argument and nominating a
+committer as the Release Manager (including themselves). There's no formal process, no vote
+requirements, and no timing requirements. Any objections should be resolved by consensus before
+starting the release.</p>
+<p><em>Checklist to proceed to next step:</em></p>
+<ol>
+<li>Community agrees to release</li>
+<li>Community selects a Release Manager</li>
+</ol>
+<h2>
+<a id="prepare-for-the-release" class="anchor" href="#prepare-for-the-release" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prepare for the release</h2>
+<p>Before your first release, you should perform one-time configuration steps. This will set up your
+security keys for signing the artifacts and access release repository.</p>
+<p>To prepare for each release, you should audit the project status in the Jira, and do necessary
+bookkeeping. Finally, you should tag a release.</p>
+<h3>
+<a id="one-time-setup-instructions" class="anchor" href="#one-time-setup-instructions" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>One-time setup instructions</h3>
+<h4>
+<a id="gpg-key" class="anchor" href="#gpg-key" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>GPG Key</h4>
+<p>You need to have a GPG key to sign the release artifacts. Please be aware of the ASF-wide
+<a href="https://www.apache.org/dev/release-signing.html" target=_blank rel="nofollow">release signing guidelines</a>. If you don't have
+a GPG key associated with your Apache account, please create one according to the guidelines.</p>
+<p>Determine your Apache GPG key and key ID, as follows:</p>
+<pre><code>gpg --list-keys
+</code></pre>
+<p>This will list your GPG keys. One of these should reflect your Apache account, for example:</p>
+<pre><code>pub   2048R/845E6689 2016-02-23
+uid                  Nomen Nescio &lt;anonymous@apache.org&gt;
+sub   2048R/BA4D50BE 2016-02-23
+</code></pre>
+<p>Here, the key ID is the 8-digit hex string in the <code>pub</code> line: <code>845E6689</code>.</p>
+<p>Now, add your Apache GPG key to the CarbonData's <code>KEYS</code> file in <code>dev</code> and <code>release</code> repositories
+at <code>dist.apache.org</code>. Follow the instructions listed at the top of these files.</p>
+<p>Configure <code>git</code> to use this key when signing code by giving it your key ID, as follows:</p>
+<pre><code>git config --global user.signingkey 845E6689
+</code></pre>
+<p>You may drop the <code>--global</code> option if you'd prefer to use this key for the current repository only.</p>
+<p>You may wish to start <code>gpg-agent</code> to unlock your GPG key only once using your passphrase.
+Otherwise, you may need to enter this passphrase several times. The setup of <code>gpg-agent</code> varies
+based on operating system, but may be something like this:</p>
+<pre><code>eval $(gpg-agent --daemon --no-grab --write-env-file $HOME/.gpg-agent-info)
+export GPG_TTY=$(tty)
+export GPG_AGENT_INFO
+</code></pre>
+<h4>
+<a id="access-to-apache-nexus" class="anchor" href="#access-to-apache-nexus" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Access to Apache Nexus</h4>
+<p>Configure access to the <a href="https://repository.apache.org" target=_blank rel="nofollow">Apache Nexus repository</a>, used for
+staging repository and promote the artifacts to Maven Central.</p>
+<ol>
+<li>You log in with your Apache account.</li>
+<li>Confirm you have appropriate access by finding <code>org.apache.carbondata</code> under <code>Staging Profiles</code>.</li>
+<li>Navigate to your <code>Profile</code> (top right dropdown menu of the page).</li>
+<li>Choose <code>User Token</code> from the dropdown, then click <code>Access User Token</code>. Copy a snippet of the
+Maven XML configuration block.</li>
+<li>Insert this snippet twice into your global Maven <code>settings.xml</code> file, typically <code>${HOME]/ .m2/settings.xml</code>. The end result should look like this, where <code>TOKEN_NAME</code> and <code>TOKEN_PASSWORD</code>
+are your secret tokens:</li>
+</ol>
+<pre><code> &lt;settings&gt;
+   &lt;servers&gt;
+     &lt;server&gt;
+       &lt;id&gt;apache.releases.https&lt;/id&gt;
+       &lt;username&gt;TOKEN_NAME&lt;/username&gt;
+       &lt;password&gt;TOKEN_PASSWORD&lt;/password&gt;
+     &lt;/server&gt;
+     &lt;server&gt;
+       &lt;id&gt;apache.snapshots.https&lt;/id&gt;
+       &lt;username&gt;TOKEN_NAME&lt;/username&gt;
+       &lt;password&gt;TOKEN_PASSWORD&lt;/password&gt;
+     &lt;/server&gt;
+   &lt;/servers&gt;
+ &lt;/settings&gt;
+</code></pre>
+<h4>
+<a id="create-a-new-version-in-jira" class="anchor" href="#create-a-new-version-in-jira" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create a new version in Jira</h4>
+<p>When contributors resolve an issue in Jira, they are tagging it with a release that will contain
+their changes. With the release currently underway, new issues should be resolved against a
+subsequent future release. Therefore, you should create a release item for this subsequent
+release, as follows:</p>
+<ol>
+<li>In Jira, navigate to <code>CarbonData &gt; Administration &gt; Versions</code>.</li>
+<li>Add a new release: choose the next minor version number compared to the one currently
+underway, select today's date as the <code>Start Date</code>, and choose <code>Add</code>.</li>
+</ol>
+<h4>
+<a id="triage-release-blocking-issues-in-jira" class="anchor" href="#triage-release-blocking-issues-in-jira" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Triage release-blocking issues in Jira</h4>
+<p>There could be outstanding release-blocking issues, which should be triaged before proceeding to
+build the release. We track them by assigning a specific <code>Fix Version</code> field even before the
+issue is resolved.</p>
+<p>The list of release-blocking issues is available at the <a href="https://issues.apache.org/jira/browse/CARBONDATA/?selectedTab=com.atlassian.jira.jira-projects-plugin:versions-panel" rel="nofollow">version status page</a>.
+Triage each unresolved issue with one of the following resolutions:</p>
+<ul>
+<li>If the issue has been resolved and Jira was not updated, resolve it accordingly.</li>
+<li>If the issue has not been resolved and it is acceptable to defer until the next release, update
+the <code>Fix Version</code> field to the new version you just created. Please consider discussing this
+with stakeholders and the dev@ mailing list, as appropriate.</li>
+<li>If the issue has not been resolved and it is not acceptable to release until it is fixed, the
+release cannot proceed. Instead, work with the CarbonData community to resolve the issue.</li>
+</ul>
+<h4>
+<a id="review-release-notes-in-jira" class="anchor" href="#review-release-notes-in-jira" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Review Release Notes in Jira</h4>
+<p>Jira automatically generates Release Notes based on the <code>Fix Version</code> applied to the issues.
+Release Notes are intended for CarbonData users (not CarbonData committers/contributors). You
+should ensure that Release Notes are informative and useful.</p>
+<p>Open the release notes from the <a href="https://issues.apache.org/jira/browse/CARBONDATA/?selectedTab=com.atlassian.jira.jira-projects-plugin:versions-panel" rel="nofollow">version status page</a>
+by choosing the release underway and clicking Release Notes.</p>
+<p>You should verify that the issues listed automatically by Jira are appropriate to appear in the
+Release Notes. Specifically, issues should:</p>
+<ul>
+<li>Be appropriate classified as <code>Bug</code>, <code>New Feature</code>, <code>Improvement</code>, etc.</li>
+<li>Represent noteworthy user-facing changes, such as new functionality, backward-incompatible
+changes, or performance improvements.</li>
+<li>Have occurred since the previous release; an issue that was introduced and fixed between
+releases should not appear in the Release Notes.</li>
+<li>Have an issue title that makes sense when read on its own.</li>
+</ul>
+<p>Adjust any of the above properties to the improve clarity and presentation of the Release Notes.</p>
+<h4>
+<a id="verify-that-a-release-build-works" class="anchor" href="#verify-that-a-release-build-works" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Verify that a Release Build works</h4>
+<p>Run <code>mvn clean install -Prelease</code> to ensure that the build processes that are specific to that
+profile are in good shape.</p>
+<p><em>Checklist to proceed to the next step:</em></p>
+<ol>
+<li>Release Manager's GPG key is published to <code>dist.apache.org</code>.</li>
+<li>Release Manager's GPG key is configured in <code>git</code> configuration.</li>
+<li>Release Manager has <code>org.apache.carbondata</code> listed under <code>Staging Profiles</code> in Nexus.</li>
+<li>Release Manager's Nexus User Token is configured in <code>settings.xml</code>.</li>
+<li>Jira release item for the subsequent release has been created.</li>
+<li>There are no release blocking Jira issues.</li>
+<li>Release Notes in Jira have been audited and adjusted.</li>
+</ol>
+<h3>
+<a id="build-a-release" class="anchor" href="#build-a-release" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Build a release</h3>
+<p>Use Maven release plugin to tag and build release artifacts, as follows:</p>
+<pre><code>mvn release:prepare
+</code></pre>
+<p>Use Maven release plugin to stage these artifacts on the Apache Nexus repository, as follows:</p>
+<pre><code>mvn release:perform
+</code></pre>
+<p>Review all staged artifacts. They should contain all relevant parts for each module, including
+<code>pom.xml</code>, jar, test jar, source, etc. Artifact names should follow
+<a href="https://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.carbondata%22" rel="nofollow">the existing format</a>
+in which artifact name mirrors directory structure. Carefully review any new artifacts.</p>
+<p>Close the staging repository on Nexus. When prompted for a description, enter "Apache CarbonData
+x.x.x release".</p>
+<h3>
+<a id="stage-source-release-on-distapacheorg" class="anchor" href="#stage-source-release-on-distapacheorg" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Stage source release on dist.apache.org</h3>
+<p>Copy the source release to dev repository on <code>dist.apache.org</code>.</p>
+<ol>
+<li>If you have not already, check out the section of the <code>dev</code> repository on <code>dist.apache.org</code> via Subversion. In a fresh directory:</li>
+</ol>
+<pre><code>svn co https://dist.apache.org/repos/dist/dev/carbondata
+</code></pre>
+<ol start="2">
+<li>Make a directory for the new release:</li>
+</ol>
+<pre><code>mkdir x.x.x
+</code></pre>
+<ol start="3">
+<li>Copy the CarbonData source distribution, hash, and GPG signature:</li>
+</ol>
+<pre><code>cp apache-carbondata-x.x.x-source-release.zip x.x.x
+</code></pre>
+<ol start="4">
+<li>Add and commit the files:</li>
+</ol>
+<pre><code>svn add x.x.x
+svn commit
+</code></pre>
+<ol start="5">
+<li>Verify the files are <a href="https://dist.apache.org/repos/dist/dev/carbondata" target=_blank rel="nofollow">present</a>.</li>
+</ol>
+<h3>
+<a id="propose-a-pull-request-for-website-updates" class="anchor" href="#propose-a-pull-request-for-website-updates" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Propose a pull request for website updates</h3>
+<p>The final step of building a release candidate is to propose a website pull request.</p>
+<p>This pull request should update the following page with the new release:</p>
+<ul>
+<li><code>src/main/webapp/index.html</code></li>
+<li><code>src/main/webapp/docs/latest/mainpage.html</code></li>
+</ul>
+<p><em>Checklist to proceed to the next step:</em></p>
+<ol>
+<li>Maven artifacts deployed to the staging repository of
+<a href="https://repository.apache.org" target=_blank rel="nofollow">repository.apache.org</a>
+</li>
+<li>Source distribution deployed to the dev repository of
+<a href="https://dist.apache.org/repos/dist/dev/carbondata/" target=_blank rel="nofollow">dist.apache.org</a>
+</li>
+<li>Website pull request to list the release.</li>
+</ol>
+<h2>
+<a id="vote-on-the-release-candidate" class="anchor" href="#vote-on-the-release-candidate" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Vote on the release candidate</h2>
+<p>Once you have built and individually reviewed the release candidate, please share it for the
+community-wide review. Please review foundation-wide <a href="http://www.apache.org/foundation/voting.html" target=_blank rel="nofollow">voting guidelines</a>
+for more information.</p>
+<p>Start the review-and-vote thread on the dev@ mailing list. Here's an email template; please
+adjust as you see fit:</p>
+<pre><code>From: Release Manager
+To: dev@carbondata.apache.org
+Subject: [VOTE] Apache CarbonData Release x.x.x
+
+Hi everyone,
+Please review and vote on the release candidate for the version x.x.x, as follows:
+
+[ ] +1, Approve the release
+[ ] -1, Do not approve the release (please provide specific comments)
+
+The complete staging area is available for your review, which includes:
+* JIRA release notes [1],
+* the official Apache source release to be deployed to dist.apache.org [2], which is signed with the key with fingerprint FFFFFFFF [3],
+* all artifacts to be deployed to the Maven Central Repository [4],
+* source code tag "x.x.x" [5],
+* website pull request listing the release [6].
+
+The vote will be open for at least 72 hours. It is adopted by majority approval, with at least 3 PMC affirmative votes.
+
+Thanks,
+Release Manager
+
+[1] link
+[2] link
+[3] https://dist.apache.org/repos/dist/dist/carbondata/KEYS
+[4] link
+[5] link
+[6] link
+</code></pre>
+<p>If there are any issues found in the release candidate, reply on the vote thread to cancel the vote.
+There?s no need to wait 72 hours. Proceed to the <code>Cancel a Release (Fix Issues)</code> step below and
+address the problem.
+However, some issues don?t require cancellation.
+For example, if an issue is found in the website pull request, just correct it on the spot and the
+vote can continue as-is.</p>
+<p>If there are no issues, reply on the vote thread to close the voting. Then, tally the votes in a
+separate email. Here?s an email template; please adjust as you see fit.</p>
+<pre><code>From: Release Manager
+To: dev@carbondata.apache.org
+Subject: [RESULT][VOTE] Apache CarbonData Release x.x.x
+
+I'm happy to announce that we have unanimously approved this release.
+
+There are XXX approving votes, XXX of which are binding:
+* approver 1
+* approver 2
+* approver 3
+* approver 4
+
+There are no disapproving votes.
+
+Thanks everyone!
+</code></pre>
+<p>While in incubation, the Apache Incubator PMC must also vote on each release, using the same
+process as above. Start the review and vote thread on the <code>general@incubator.apache.org</code> list.</p>
+<p><em>Checklist to proceed to the final step:</em></p>
+<ol>
+<li>Community votes to release the proposed release</li>
+<li>While in incubation, Apache Incubator PMC votes to release the proposed release</li>
+</ol>
+<h2>
+<a id="cancel-a-release-fix-issues" class="anchor" href="#cancel-a-release-fix-issues" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Cancel a Release (Fix Issues)</h2>
+<p>Any issue identified during the community review and vote should be fixed in this step.</p>
+<p>To fully cancel a vote:</p>
+<ul>
+<li>Cancel the current release and verify the version is back to the correct SNAPSHOT:</li>
+</ul>
+<pre><code>mvn release:cancel
+</code></pre>
+<ul>
+<li>Drop the release tag:</li>
+</ul>
+<pre><code>git tag -d x.x.x
+git push --delete apache x.x.x
+</code></pre>
+<ul>
+<li>Drop the staging repository on Nexus (<a href="https://repository.apache.org" target=_blank rel="nofollow">repository.apache.org</a>)</li>
+</ul>
+<p>Verify the version is back to the correct SNAPSHOT.</p>
+<p>Code changes should be proposed as standard pull requests and merged.</p>
+<p>Once all issues have been resolved, you should go back and build a new release candidate with
+these changes.</p>
+<h2>
+<a id="finalize-the-release" class="anchor" href="#finalize-the-release" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Finalize the release</h2>
+<p>Once the release candidate has been reviewed and approved by the community, the release should be
+finalized. This involves the final deployment of the release to the release repositories,
+merging the website changes, and announce the release.</p>
+<h3>
+<a id="deploy-artifacts-to-maven-central-repository" class="anchor" href="#deploy-artifacts-to-maven-central-repository" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deploy artifacts to Maven Central repository</h3>
+<p>On Nexus, release the staged artifacts to Maven Central repository. In the <code>Staging Repositories</code>
+section, find the relevant release candidate <code>orgapachecarbondata-XXX</code> entry and click <code>Release</code>.</p>
+<h3>
+<a id="deploy-source-release-to-distapacheorg" class="anchor" href="#deploy-source-release-to-distapacheorg" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deploy source release to dist.apache.org</h3>
+<p>Copy the source release from the <code>dev</code> repository to <code>release</code> repository at <code>dist.apache.org</code>
+using Subversion.</p>
+<h3>
+<a id="merge-website-pull-request" class="anchor" href="#merge-website-pull-request" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Merge website pull request</h3>
+<p>Merge the website pull request to list the release created earlier.</p>
+<h3>
+<a id="mark-the-version-as-released-in-jira" class="anchor" href="#mark-the-version-as-released-in-jira" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Mark the version as released in Jira</h3>
+<p>In Jira, inside <a href="https://issues.apache.org/jira/plugins/servlet/project-config/CARBONDATA/versions" target=_blank rel="nofollow">version management</a>
+, hover over the current release and a settings menu will appear. Click <code>Release</code>, and select
+today's state.</p>
+<p><em>Checklist to proceed to the next step:</em></p>
+<ol>
+<li>Maven artifacts released and indexed in the
+<a href="https://search.maven.org/#search%7Cga%7C1%7Cg%3A%22org.apache.carbondata%22" rel="nofollow">Maven Central repository</a>
+</li>
+<li>Source distribution available in the release repository of
+<a href="https://dist.apache.org/repos/dist/release/carbondata/" target=_blank rel="nofollow">dist.apache.org</a>
+</li>
+<li>Website pull request to list the release merged</li>
+<li>Release version finalized in Jira</li>
+</ol>
+<h2>
+<a id="promote-the-release" class="anchor" href="#promote-the-release" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Promote the release</h2>
+<p>Once the release has been finalized, the last step of the process is to promote the release
+within the project and beyond.</p>
+<h3>
+<a id="apache-mailing-lists" class="anchor" href="#apache-mailing-lists" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Apache mailing lists</h3>
+<p>Announce on the dev@ mailing list that the release has been finished.</p>
+<p>Announce on the user@ mailing list that the release is available, listing major improvements and
+contributions.</p>
+<p>While in incubation, announce the release on the Incubator's general@ mailing list.</p>
+<p><em>Checklist to declare the process completed:</em></p>
+<ol>
+<li>Release announced on the user@ mailing list.</li>
+<li>Release announced on the Incubator's general@ mailing list.</li>
+<li>Completion declared on the dev@ mailing list.</li>
+</ol>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__release').addClass('selected'); });
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/s3-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/s3-guide.html b/src/main/webapp/s3-guide.html
index d9edeaf..9042b19 100644
--- a/src/main/webapp/s3-guide.html
+++ b/src/main/webapp/s3-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,14 +153,65 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
                             <div class="row">
                                 <div class="col-sm-12  col-md-12">
                                     <div>
-<p>#S3 Guide (Alpha Feature 1.4.1)</p>
+<h1>
+<a id="s3-guide-alpha-feature-141" class="anchor" href="#s3-guide-alpha-feature-141" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>S3 Guide (Alpha Feature 1.4.1)</h1>
 <p>Object storage is the recommended storage format in cloud as it can support storing large data
 files. S3 APIs are widely used for accessing object stores. This can be
 used to store or retrieve data on Amazon cloud, Huawei Cloud(OBS) or on any other object
@@ -168,7 +220,8 @@ Storing data in cloud is advantageous as there are no restrictions on the size o
 data and the data can be accessed from anywhere at any time.
 Carbondata can support any Object Storage that conforms to Amazon S3 API.
 Carbondata relies on Hadoop provided S3 filesystem APIs to access Object stores.</p>
-<p>#Writing to Object Storage</p>
+<h1>
+<a id="writing-to-object-storage" class="anchor" href="#writing-to-object-storage" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Writing to Object Storage</h1>
 <p>To store carbondata files onto Object Store, <code>carbon.storelocation</code> property will have
 to be configured with Object Store path in CarbonProperties file.</p>
 <p>For example:</p>
@@ -180,8 +233,9 @@ table DDL command.</p>
 <p>For example:</p>
 <pre><code>CREATE TABLE IF NOT EXISTS db1.table1(col1 string, col2 int) STORED AS carbondata LOCATION 's3a://mybucket/carbonstore'
 </code></pre>
-<p>For more details on create table, Refer <a href="./data-management-on-carbondata.html#create-table">data-management-on-carbondata</a></p>
-<p>#Authentication</p>
+<p>For more details on create table, Refer <a href="ddl-of-carbondata.html#create-table">DDL of CarbonData</a></p>
+<h1>
+<a id="authentication" class="anchor" href="#authentication" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Authentication</h1>
 <p>Authentication properties will have to be configured to store the carbondata files on to S3 location.</p>
 <p>Authentication properties can be set in any of the following ways:</p>
 <ol>
@@ -211,16 +265,19 @@ spark.hadoop.fs.s3a.access.key=456
 <pre><code>sparkSession.sparkContext.hadoopConfiguration.set("fs.s3a.secret.key", "123")
 sparkSession.sparkContext.hadoopConfiguration.set("fs.s3a.access.key","456")
 </code></pre>
-<p>#Recommendations</p>
+<h1>
+<a id="recommendations" class="anchor" href="#recommendations" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Recommendations</h1>
 <ol>
 <li>Object Storage like S3 does not support file leasing mechanism(supported by HDFS) that is
 required to take locks which ensure consistency between concurrent operations therefore, it is
-recommended to set the configurable lock path property(<a href="https://github.com/apache/carbondata/blob/master/docs/configuration-parameters.html#miscellaneous-configuration">carbon.lock.path</a>)
+recommended to set the configurable lock path property(<a href="./configuration-parameters.html#system-configuration">carbon.lock.path</a>)
 to a HDFS directory.</li>
-<li>Concurrent data manipulation operations are not supported. Object stores follow eventual
-consistency semantics, i.e., any put request might take some time to reflect when trying to list
-.This behaviour causes not to ensure the data read is always consistent or latest.</li>
+<li>Concurrent data manipulation operations are not supported. Object stores follow eventual consistency semantics, i.e., any put request might take some time to reflect when trying to list. This behaviour causes the data read is always not consistent or not the latest.</li>
 </ol>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__s3').addClass('selected'); });
+</script>
 </div>
 </div>
 </div>
@@ -236,4 +293,4 @@ consistency semantics, i.e., any put request might take some time to reflect whe
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/sdk-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/sdk-guide.html b/src/main/webapp/sdk-guide.html
index cc55dc0..95ae214 100644
--- a/src/main/webapp/sdk-guide.html
+++ b/src/main/webapp/sdk-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -502,6 +553,22 @@ public CarbonWriterBuilder taskNo(long taskNo);
 public CarbonWriterBuilder withLoadOptions(Map&lt;String, String&gt; options);
 </code></pre>
 <pre><code>/**
+ * To support the table properties for sdk writer
+ *
+ * @param options key,value pair of create table properties.
+ * supported keys values are
+ * a. blocksize -- [1-2048] values in MB. Default value is 1024
+ * b. blockletsize -- values in MB. Default value is 64 MB
+ * c. localDictionaryThreshold -- positive value, default is 10000
+ * d. enableLocalDictionary -- true / false. Default is false
+ * e. sortcolumns -- comma separated column. "c1,c2". Default all dimensions are sorted.
+ *
+ * @return updated CarbonWriterBuilder
+ */
+public CarbonWriterBuilder withTableProperties(Map&lt;String, String&gt; options);
+</code></pre>
+<pre><code>/**
+* this writer is not thread safe, use buildThreadSafeWriterForCSVInput in multi thread environment
 * Build a {@link CarbonWriter}, which accepts row in CSV format object
 * @param schema carbon Schema object {org.apache.carbondata.sdk.file.Schema}
 * @return CSVCarbonWriter
@@ -511,6 +578,19 @@ public CarbonWriterBuilder withLoadOptions(Map&lt;String, String&gt; options);
 public CarbonWriter buildWriterForCSVInput(org.apache.carbondata.sdk.file.Schema schema) throws IOException, InvalidLoadOptionException;
 </code></pre>
 <pre><code>/**
+* Can use this writer in multi-thread instance.
+* Build a {@link CarbonWriter}, which accepts row in CSV format
+* @param schema carbon Schema object {org.apache.carbondata.sdk.file.Schema}
+* @param numOfThreads number of threads() in which .write will be called.              
+* @return CSVCarbonWriter
+* @throws IOException
+* @throws InvalidLoadOptionException
+*/
+public CarbonWriter buildThreadSafeWriterForCSVInput(Schema schema, short numOfThreads)
+  throws IOException, InvalidLoadOptionException;
+</code></pre>
+<pre><code>/**
+* this writer is not thread safe, use buildThreadSafeWriterForAvroInput in multi thread environment
 * Build a {@link CarbonWriter}, which accepts Avro format object
 * @param avroSchema avro Schema object {org.apache.avro.Schema}
 * @return AvroCarbonWriter 
@@ -520,6 +600,19 @@ public CarbonWriter buildWriterForCSVInput(org.apache.carbondata.sdk.file.Schema
 public CarbonWriter buildWriterForAvroInput(org.apache.avro.Schema schema) throws IOException, InvalidLoadOptionException;
 </code></pre>
 <pre><code>/**
+* Can use this writer in multi-thread instance.
+* Build a {@link CarbonWriter}, which accepts Avro object
+* @param avroSchema avro Schema object {org.apache.avro.Schema}
+* @param numOfThreads number of threads() in which .write will be called.
+* @return AvroCarbonWriter
+* @throws IOException
+* @throws InvalidLoadOptionException
+*/
+public CarbonWriter buildThreadSafeWriterForAvroInput(org.apache.avro.Schema avroSchema, short numOfThreads)
+  throws IOException, InvalidLoadOptionException
+</code></pre>
+<pre><code>/**
+* this writer is not thread safe, use buildThreadSafeWriterForJsonInput in multi thread environment
 * Build a {@link CarbonWriter}, which accepts Json object
 * @param carbonSchema carbon Schema object
 * @return JsonCarbonWriter
@@ -528,6 +621,17 @@ public CarbonWriter buildWriterForAvroInput(org.apache.avro.Schema schema) throw
 */
 public JsonCarbonWriter buildWriterForJsonInput(Schema carbonSchema);
 </code></pre>
+<pre><code>/**
+* Can use this writer in multi-thread instance.
+* Build a {@link CarbonWriter}, which accepts Json object
+* @param carbonSchema carbon Schema object
+* @param numOfThreads number of threads() in which .write will be called.
+* @return JsonCarbonWriter
+* @throws IOException
+* @throws InvalidLoadOptionException
+*/
+public JsonCarbonWriter buildThreadSafeWriterForJsonInput(Schema carbonSchema, short numOfThreads)
+</code></pre>
 <h3>
 <a id="class-orgapachecarbondatasdkfilecarbonwriter" class="anchor" href="#class-orgapachecarbondatasdkfilecarbonwriter" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.CarbonWriter</h3>
 <pre><code>/**
@@ -536,7 +640,7 @@ public JsonCarbonWriter buildWriterForJsonInput(Schema carbonSchema);
 *                      which is one row of data.
 * If CSVCarbonWriter, object is of type String[], which is one row of data
 * If JsonCarbonWriter, object is of type String, which is one row of json
-* Note: This API is not thread safe
+* Note: This API is not thread safe if writer is not built with number of threads argument.
 * @param object
 * @throws IOException
 */
@@ -780,7 +884,6 @@ External client can make use of this reader to read CarbonData files without Car
    *
    * @param dataFilePath complete path including carbondata file name
    * @return Schema object
-   * @throws IOException
    */
   public static Schema readSchemaInDataFile(String dataFilePath);
 </code></pre>
@@ -882,7 +985,11 @@ public String getProperty(String key);
 */
 public String getProperty(String key, String defaultValue);
 </code></pre>
-<p>Reference : <a href="http://carbondata.apache.org/configuration-parameters.html" target=_blank rel="nofollow">list of carbon properties</a></p>
+<p>Reference : <a href="./configuration-parameters.html">list of carbon properties</a></p>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__api').addClass('selected'); });
+</script>
 </div>
 </div>
 </div>
@@ -898,4 +1005,4 @@ public String getProperty(String key, String defaultValue);
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/sdk-writer-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/sdk-writer-guide.html b/src/main/webapp/sdk-writer-guide.html
index 36bb9ad..5553194 100644
--- a/src/main/webapp/sdk-writer-guide.html
+++ b/src/main/webapp/sdk-writer-guide.html
@@ -546,4 +546,4 @@ public static org.apache.carbondata.sdk.file.Schema getCarbonSchemaFromAvroSchem
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/security.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/security.html b/src/main/webapp/security.html
index 9168fa3..2aa4b8e 100644
--- a/src/main/webapp/security.html
+++ b/src/main/webapp/security.html
@@ -205,4 +205,4 @@
 <script src="js/custom.js"></script>
 <script src="js/mdNavigation.js" type="text/javascript"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/segment-management-on-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/segment-management-on-carbondata.html b/src/main/webapp/segment-management-on-carbondata.html
new file mode 100644
index 0000000..528ee9d
--- /dev/null
+++ b/src/main/webapp/segment-management-on-carbondata.html
@@ -0,0 +1,326 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h2>
+<a id="segment-management" class="anchor" href="#segment-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SEGMENT MANAGEMENT</h2>
+<p>Each load into CarbonData is written into a separate folder called Segment.Segments is a powerful
+concept which helps to maintain consistency of data and easy transaction management.CarbonData provides DML (Data Manipulation Language) commands to maintain the segments.</p>
+<ul>
+<li><a href="#show-segment">Show Segments</a></li>
+<li><a href="#delete-segment-by-id">Delete Segment by ID</a></li>
+<li><a href="#delete-segment-by-date">Delete Segment by Date</a></li>
+<li><a href="#query-data-with-specified-segments">Query Data with Specified Segments</a></li>
+</ul>
+<h3>
+<a id="show-segment" class="anchor" href="#show-segment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW SEGMENT</h3>
+<p>This command is used to list the segments of CarbonData table.</p>
+<pre><code>SHOW [HISTORY] SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
+</code></pre>
+<p>Example:
+Show visible segments</p>
+<pre><code>SHOW SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4
+</code></pre>
+<p>Show all segments, include invisible segments</p>
+<pre><code>SHOW HISTORY SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4
+</code></pre>
+<h3>
+<a id="delete-segment-by-id" class="anchor" href="#delete-segment-by-id" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY ID</h3>
+<p>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.</p>
+<p>The following command will get the segmentID.</p>
+<pre><code>SHOW SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
+</code></pre>
+<p>After you retrieve the segment ID of the segment that you want to delete, execute the following command to delete the selected segment.</p>
+<pre><code>DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.ID IN (segment_id1, segments_id2, ...)
+</code></pre>
+<p>Example:</p>
+<pre><code>DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0)
+DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0,5,8)
+</code></pre>
+<h3>
+<a id="delete-segment-by-date" class="anchor" href="#delete-segment-by-date" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY DATE</h3>
+<p>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.</p>
+<pre><code>DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.STARTTIME BEFORE DATE_VALUE
+</code></pre>
+<p>Example:</p>
+<pre><code>DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.STARTTIME BEFORE '2017-06-01 12:05:06' 
+</code></pre>
+<h3>
+<a id="query-data-with-specified-segments" class="anchor" href="#query-data-with-specified-segments" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>QUERY DATA WITH SPECIFIED SEGMENTS</h3>
+<p>This command is used to read data from specified segments during CarbonScan.</p>
+<p>Get the Segment ID:</p>
+<pre><code>SHOW SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
+</code></pre>
+<p>Set the segment IDs for table</p>
+<pre><code>SET carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt; = &lt;list of segment IDs&gt;
+</code></pre>
+<p><strong>NOTE:</strong>
+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.</p>
+<p>If user wants to query with segments reading in multi threading mode, then CarbonSession. threadSet can be used instead of SET query.</p>
+<pre><code>CarbonSession.threadSet ("carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt;","&lt;list of segment IDs&gt;");
+</code></pre>
+<p>Reset the segment IDs</p>
+<pre><code>SET carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt; = *;
+</code></pre>
+<p>If user wants to query with segments reading in multi threading mode, then CarbonSession. threadSet can be used instead of SET query.</p>
+<pre><code>CarbonSession.threadSet ("carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt;","*");
+</code></pre>
+<p><strong>Examples:</strong></p>
+<ul>
+<li>Example to show the list of segment IDs,segment status, and other required details and then specify the list of segments to be read.</li>
+</ul>
+<pre><code>SHOW SEGMENTS FOR carbontable1;
+
+SET carbon.input.segments.db.carbontable1 = 1,3,9;
+</code></pre>
+<ul>
+<li>Example to query with segments reading in multi threading mode:</li>
+</ul>
+<pre><code>CarbonSession.threadSet ("carbon.input.segments.db.carbontable_Multi_Thread","1,3");
+</code></pre>
+<ul>
+<li>Example for threadset in multithread environment (following shows how it is used in Scala code):</li>
+</ul>
+<pre><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();
+   }
+ }
+</code></pre>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/streaming-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/streaming-guide.html b/src/main/webapp/streaming-guide.html
index 6e2e7ee..c6d8391 100644
--- a/src/main/webapp/streaming-guide.html
+++ b/src/main/webapp/streaming-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -432,6 +483,17 @@ streaming table using following DDL.</p>
 <li>block delete "streaming" segment while the streaming ingestion is running.</li>
 <li>block drop the streaming table while the streaming ingestion is running.</li>
 </ol>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -447,4 +509,4 @@ streaming table using following DDL.</p>
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>


[33/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/data-management-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/data-management-on-carbondata.html b/content/data-management-on-carbondata.html
deleted file mode 100644
index bb5ae78..0000000
--- a/content/data-management-on-carbondata.html
+++ /dev/null
@@ -1,1321 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
-                                   target="_blank">Apache CarbonData 1.4.1</a></li>
-							<li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
-                                   target="_blank">Apache CarbonData 1.4.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
-                                   target="_blank">Apache CarbonData 1.3.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
-                                   target="_blank">Apache CarbonData 1.3.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search" target="_blank">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input"  placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="row">
-                                <div class="col-sm-12  col-md-12">
-                                    <div>
-<h1>
-<a id="data-management-on-carbondata" class="anchor" href="#data-management-on-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Management on CarbonData</h1>
-<p>This tutorial is going to introduce all commands and data operations on CarbonData.</p>
-<ul>
-<li><a href="#create-table">CREATE TABLE</a></li>
-<li><a href="#create-database">CREATE DATABASE</a></li>
-<li><a href="#table-management">TABLE MANAGEMENT</a></li>
-<li><a href="#load-data">LOAD DATA</a></li>
-<li><a href="#update-and-delete">UPDATE AND DELETE</a></li>
-<li><a href="#compaction">COMPACTION</a></li>
-<li><a href="#partition">PARTITION</a></li>
-<li><a href="#bucketing">BUCKETING</a></li>
-<li><a href="#segment-management">SEGMENT MANAGEMENT</a></li>
-</ul>
-<h2>
-<a id="create-table" class="anchor" href="#create-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE</h2>
-<p>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.</p>
-<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type , ...)]
-STORED AS carbondata
-[TBLPROPERTIES (property_name=property_value, ...)]
-[LOCATION 'path']
-</code></pre>
-<p><strong>NOTE:</strong> CarbonData also supports "STORED AS carbondata" and "USING carbondata". Find example code at <a href="https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala" target=_blank>CarbonSessionExample</a> in the CarbonData repo.</p>
-<h3>
-<a id="usage-guidelines" class="anchor" href="#usage-guidelines" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Usage Guidelines</h3>
-<p>Following are the guidelines for TBLPROPERTIES, CarbonData's additional table options can be set via carbon.properties.</p>
-<ul>
-<li>
-<p><strong>Dictionary Encoding Configuration</strong></p>
-<p>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.</p>
-<pre><code>TBLPROPERTIES ('DICTIONARY_INCLUDE'='column1, column2')
-</code></pre>
-<p>NOTE: Dictionary Include/Exclude for complex child columns is not supported.</p>
-</li>
-<li>
-<p><strong>Inverted Index Configuration</strong></p>
-<p>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.</p>
-<pre><code>TBLPROPERTIES ('NO_INVERTED_INDEX'='column1, column3')
-</code></pre>
-</li>
-<li>
-<p><strong>Sort Columns Configuration</strong></p>
-<p>This property is for users to specify which columns belong to the MDK(Multi-Dimensions-Key) index.</p>
-<ul>
-<li>If users don't specify "SORT_COLUMN" property, by default MDK index be built by using all dimension columns except complex data type column.</li>
-<li>If this property is specified but with empty argument, then the table will be loaded without sort.</li>
-<li>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.</li>
-</ul>
-<pre><code>TBLPROPERTIES ('SORT_COLUMNS'='column1, column3')
-OR
-TBLPROPERTIES ('SORT_COLUMNS'='')
-</code></pre>
-<p>NOTE: Sort_Columns for Complex datatype columns is not supported.</p>
-</li>
-<li>
-<p><strong>Sort Scope Configuration</strong></p>
-<p>This property is for users to specify the scope of the sort during data load, following are the types of sort scope.</p>
-<ul>
-<li>LOCAL_SORT: It is the default sort scope.</li>
-<li>NO_SORT: It will load the data in unsorted manner, it will significantly increase load performance.</li>
-<li>BATCH_SORT: It increases the load performance but decreases the query performance if identified blocks &gt; parallelism.</li>
-<li>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.</li>
-</ul>
-</li>
-</ul>
-<pre><code>### Example:
-</code></pre>
-<pre><code> 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 ('SORT_COLUMNS'='productName,storeCity',
-                'SORT_SCOPE'='NO_SORT')
-</code></pre>
-<p><strong>NOTE:</strong> CarbonData also supports "using carbondata". Find example code at <a href="https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/SparkSessionExample.scala" target=_blank>SparkSessionExample</a> in the CarbonData repo.</p>
-<ul>
-<li>
-<p><strong>Table Block Size Configuration</strong></p>
-<p>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.</p>
-<pre><code>TBLPROPERTIES ('TABLE_BLOCKSIZE'='512')
-</code></pre>
-<p><strong>NOTE:</strong> 512 or 512M both are accepted.</p>
-</li>
-<li>
-<p><strong>Table Compaction Configuration</strong></p>
-<p>These properties are table level compaction configurations, if not specified, system level configurations in carbon.properties will be used.
-Following are 5 configurations:</p>
-<ul>
-<li>MAJOR_COMPACTION_SIZE: same meaning as carbon.major.compaction.size, size in MB.</li>
-<li>AUTO_LOAD_MERGE: same meaning as carbon.enable.auto.load.merge.</li>
-<li>COMPACTION_LEVEL_THRESHOLD: same meaning as carbon.compaction.level.threshold.</li>
-<li>COMPACTION_PRESERVE_SEGMENTS: same meaning as carbon.numberof.preserve.segments.</li>
-<li>ALLOWED_COMPACTION_DAYS: same meaning as carbon.allowed.compaction.days.</li>
-</ul>
-<pre><code>TBLPROPERTIES ('MAJOR_COMPACTION_SIZE'='2048',
-               'AUTO_LOAD_MERGE'='true',
-               'COMPACTION_LEVEL_THRESHOLD'='5,6',
-               'COMPACTION_PRESERVE_SEGMENTS'='10',
-               'ALLOWED_COMPACTION_DAYS'='5')
-</code></pre>
-</li>
-<li>
-<p><strong>Streaming</strong></p>
-<p>CarbonData supports streaming ingestion for real-time data. You can create the ?streaming? table using the following table properties.</p>
-<pre><code>TBLPROPERTIES ('streaming'='true')
-</code></pre>
-</li>
-<li>
-<p><strong>Local Dictionary Configuration</strong></p>
-</li>
-</ul>
-<p>Columns for which dictionary is not generated needs more storage space and in turn more IO. Also since more data will have to be read during query, query performance also would suffer.Generating dictionary per blocklet for such columns would help in saving storage space and assist in improving query performance as carbondata is optimized for handling dictionary encoded columns more effectively.Generating dictionary internally per blocklet is termed as local dictionary. Please refer to <a href="../file-structure-of-carbondata.html">File structure of Carbondata</a> for understanding about the file structure of carbondata and meaning of terms like blocklet.</p>
-<p>Local Dictionary helps in:</p>
-<ol>
-<li>Getting more compression.</li>
-<li>Filter queries and full scan queries will be faster as filter will be done on encoded data.</li>
-<li>Reducing the store size and memory footprint as only unique values will be stored as part of local dictionary and corresponding data will be stored as encoded data.</li>
-<li>Getting higher IO throughput.</li>
-</ol>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>
-<p>Following Data Types are Supported for Local Dictionary:</p>
-<ul>
-<li>STRING</li>
-<li>VARCHAR</li>
-<li>CHAR</li>
-</ul>
-</li>
-<li>
-<p>Following Data Types are not Supported for Local Dictionary:</p>
-<ul>
-<li>SMALLINT</li>
-<li>INTEGER</li>
-<li>BIGINT</li>
-<li>DOUBLE</li>
-<li>DECIMAL</li>
-<li>TIMESTAMP</li>
-<li>DATE</li>
-<li>BOOLEAN</li>
-</ul>
-</li>
-<li>
-<p>In case of multi-level complex dataType columns, primitive string/varchar/char columns are considered for local dictionary generation.</p>
-</li>
-</ul>
-<p>Local dictionary will have to be enabled explicitly during create table or by enabling the system property 'carbon.local.dictionary.enable'. By default, Local Dictionary will be disabled for the carbondata table.</p>
-<p>Local Dictionary can be configured using the following properties during create table command:</p>
-<table>
-<thead>
-<tr>
-<th>Properties</th>
-<th>Default value</th>
-<th>Description</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>LOCAL_DICTIONARY_ENABLE</td>
-<td>false</td>
-<td>Whether to enable local dictionary generation. <strong>NOTE:</strong> If this property is defined, it will override the value configured at system level by 'carbon.local.dictionary.enable'</td>
-</tr>
-<tr>
-<td>LOCAL_DICTIONARY_THRESHOLD</td>
-<td>10000</td>
-<td>The maximum cardinality of a column upto which carbondata can try to generate local dictionary (maximum - 100000)</td>
-</tr>
-<tr>
-<td>LOCAL_DICTIONARY_INCLUDE</td>
-<td>string/varchar/char columns</td>
-<td>Columns for which Local Dictionary has to be generated.<strong>NOTE:</strong> Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.</td>
-</tr>
-<tr>
-<td>LOCAL_DICTIONARY_EXCLUDE</td>
-<td>none</td>
-<td>Columns for which Local Dictionary need not be generated.</td>
-</tr>
-</tbody>
-</table>
-<p><strong>Fallback behavior:</strong></p>
-<ul>
-<li>When the cardinality of a column exceeds the threshold, it triggers a fallback and the generated dictionary will be reverted and data loading will be continued without dictionary encoding.</li>
-</ul>
-<p><strong>NOTE:</strong> When fallback is triggered, the data loading performance will decrease as encoded data will be discarded and the actual data is written to the temporary sort files.</p>
-<p><strong>Points to be noted:</strong></p>
-<ol>
-<li>
-<p>Reduce Block size:</p>
-<p>Number of Blocks generated is less in case of Local Dictionary as compression ratio is high. This may reduce the number of tasks launched during query, resulting in degradation of query performance if the pruned blocks are less compared to the number of parallel tasks which can be run. So it is recommended to configure smaller block size which in turn generates more number of blocks.</p>
-</li>
-<li>
-<p>All the page-level data for a blocklet needs to be maintained in memory until all the pages encoded for local dictionary is processed in order to handle fallback. Hence the memory required for local dictionary based table is more and this memory increase is proportional to number of columns.</p>
-</li>
-</ol>
-<h3>
-<a id="example" class="anchor" href="#example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example:</h3>
-<pre><code>CREATE TABLE carbontable(
-          
-            column1 string,
-          
-            column2 string,
-          
-            column3 LONG )
-          
-  STORED BY 'carbondata'
-  TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='true','LOCAL_DICTIONARY_THRESHOLD'='1000',
-  'LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>We recommend to use Local Dictionary when cardinality is high but is distributed across multiple loads</li>
-<li>On a large cluster, decoding data can become a bottleneck for global dictionary as there will be many remote reads. In this scenario, it is better to use Local Dictionary.</li>
-<li>When cardinality is less, but loads are repetitive, it is better to use global dictionary as local dictionary generates multiple dictionary files at blocklet level increasing redundancy.</li>
-</ul>
-<ul>
-<li>
-<p><strong>Caching Min/Max Value for Required Columns</strong>
-By default, CarbonData caches min and max values of all the columns in schema.  As the load increases, the memory required to hold the min and max values increases considerably. This feature enables you to configure min and max values only for the required columns, resulting in optimized memory usage.</p>
-<p>Following are the valid values for COLUMN_META_CACHE:</p>
-<ul>
-<li>If you want no column min/max values to be cached in the driver.</li>
-</ul>
-<pre><code>COLUMN_META_CACHE=??
-</code></pre>
-<ul>
-<li>If you want only col1 min/max values to be cached in the driver.</li>
-</ul>
-<pre><code>COLUMN_META_CACHE=?col1?
-</code></pre>
-<ul>
-<li>If you want min/max values to be cached in driver for all the specified columns.</li>
-</ul>
-<pre><code>COLUMN_META_CACHE=?col1,col2,col3,??
-</code></pre>
-<p>Columns to be cached can be specified either while creating table or after creation of the table.
-During create table operation; specify the columns to be cached in table properties.</p>
-<p>Syntax:</p>
-<pre><code>CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,?) STORED BY ?carbondata? TBLPROPERTIES (?COLUMN_META_CACHE?=?col1,col2,??)
-</code></pre>
-<p>Example:</p>
-<pre><code>CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES (?COLUMN_META_CACHE?=?name?)
-</code></pre>
-<p>After creation of table or on already created tables use the alter table command to configure the columns to be cached.</p>
-<p>Syntax:</p>
-<pre><code>ALTER TABLE [dbName].tableName SET TBLPROPERTIES (?COLUMN_META_CACHE?=?col1,col2,??)
-</code></pre>
-<p>Example:</p>
-<pre><code>ALTER TABLE employee SET TBLPROPERTIES (?COLUMN_META_CACHE?=?city?)
-</code></pre>
-</li>
-<li>
-<p><strong>Caching at Block or Blocklet Level</strong></p>
-<p>This feature allows you to maintain the cache at Block level, resulting in optimized usage of the memory. The memory consumption is high if the Blocklet level caching is maintained as a Block can have multiple Blocklet.</p>
-<p>Following are the valid values for CACHE_LEVEL:</p>
-<p><em>Configuration for caching in driver at Block level (default value).</em></p>
-<pre><code>CACHE_LEVEL= ?BLOCK?
-</code></pre>
-<p><em>Configuration for caching in driver at Blocklet level.</em></p>
-<pre><code>CACHE_LEVEL= ?BLOCKLET?
-</code></pre>
-<p>Cache level can be specified either while creating table or after creation of the table.
-During create table operation specify the cache level in table properties.</p>
-<p>Syntax:</p>
-<pre><code>CREATE TABLE [dbName].tableName (col1 String, col2 String, col3 int,?) STORED BY ?carbondata? TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
-</code></pre>
-<p>Example:</p>
-<pre><code>CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
-</code></pre>
-<p>After creation of table or on already created tables use the alter table command to configure the cache level.</p>
-<p>Syntax:</p>
-<pre><code>ALTER TABLE [dbName].tableName SET TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
-</code></pre>
-<p>Example:</p>
-<pre><code>ALTER TABLE employee SET TBLPROPERTIES (?CACHE_LEVEL?=?Blocklet?)
-</code></pre>
-</li>
-</ul>
-<pre><code>- **Support Flat folder same as Hive/Parquet**
-
-  This feature allows all carbondata and index files to keep directy under tablepath. Currently all carbondata/carbonindex files written under tablepath/Fact/Part0/Segment_NUM folder and it is not same as hive/parquet folder structure. This feature makes all files written will be directly under tablepath, it does not maintain any segment folder structure.This is useful for interoperability between the execution engines and plugin with other execution engines like hive or presto becomes easier.
-
-  Following table property enables this feature and default value is false.
-  ```
-   'flat_folder'='true'
-  ```
-  Example:
-  ```
-  CREATE TABLE employee (name String, city String, id int) STORED BY ?carbondata? TBLPROPERTIES ('flat_folder'='true')
-  ```
-
-- **String longer than 32000 characters**
-
- In common scenarios, the length of string is less than 32000,
- so carbondata stores the length of content using Short to reduce memory and space consumption.
- To support string longer than 32000 characters, carbondata introduces a table property called `LONG_STRING_COLUMNS`.
- For these columns, carbondata internally stores the length of content using Integer.
-
- You can specify the columns as 'long string column' using below tblProperties:
-
- ```
- // specify col1, col2 as long string columns
- TBLPROPERTIES ('LONG_STRING_COLUMNS'='col1,col2')
- ```
-
- Besides, you can also use this property through DataFrame by
- ```
- df.format("carbondata")
-   .option("tableName", "carbonTable")
-   .option("long_string_columns", "col1, col2")
-   .save()
- ```
-
- If you are using Carbon-SDK, you can specify the datatype of long string column as `varchar`.
- You can refer to SDKwriterTestCase for example.
-
- **NOTE:** The LONG_STRING_COLUMNS can only be string/char/varchar columns and cannot be dictionary_include/sort_columns/complex columns.
-</code></pre>
-<h2>
-<a id="create-table-as-select" class="anchor" href="#create-table-as-select" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE TABLE AS SELECT</h2>
-<p>This function allows user 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.</p>
-<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name 
-STORED BY 'carbondata' 
-[TBLPROPERTIES (key1=val1, key2=val2, ...)] 
-AS select_statement;
-</code></pre>
-<h3>
-<a id="examples" class="anchor" href="#examples" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Examples</h3>
-<pre><code>carbon.sql("CREATE TABLE source_table(
-                           id INT,
-                           name STRING,
-                           city STRING,
-                           age INT)
-            STORED AS parquet")
-carbon.sql("INSERT INTO source_table SELECT 1,'bob','shenzhen',27")
-carbon.sql("INSERT INTO source_table SELECT 2,'david','shenzhen',31")
-
-carbon.sql("CREATE TABLE target_table
-            STORED BY 'carbondata'
-            AS SELECT city,avg(age) FROM source_table GROUP BY city")
-            
-carbon.sql("SELECT * FROM target_table").show
-  // results:
-  //    +--------+--------+
-  //    |    city|avg(age)|
-  //    +--------+--------+
-  //    |shenzhen|    29.0|
-  //    +--------+--------+
-
-</code></pre>
-<h2>
-<a id="create-external-table" class="anchor" href="#create-external-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE EXTERNAL TABLE</h2>
-<p>This function allows user to create external table by specifying location.</p>
-<pre><code>CREATE EXTERNAL TABLE [IF NOT EXISTS] [db_name.]table_name 
-STORED BY 'carbondata' LOCATION ?$FilesPath?
-</code></pre>
-<h3>
-<a id="create-external-table-on-managed-table-data-location" class="anchor" href="#create-external-table-on-managed-table-data-location" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create external table on managed table data location.</h3>
-<p>Managed table data location provided will have both FACT and Metadata folder.
-This data can be generated by creating a normal carbon table and use this path as $FilesPath in the above syntax.</p>
-<p><strong>Example:</strong></p>
-<pre><code>sql("CREATE TABLE origin(key INT, value STRING) STORED BY 'carbondata'")
-sql("INSERT INTO origin select 100,'spark'")
-sql("INSERT INTO origin select 200,'hive'")
-// creates a table in $storeLocation/origin
-
-sql(s"""
-|CREATE EXTERNAL TABLE source
-|STORED BY 'carbondata'
-|LOCATION '$storeLocation/origin'
-""".stripMargin)
-checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin"))
-</code></pre>
-<h3>
-<a id="create-external-table-on-non-transactional-table-data-location" class="anchor" href="#create-external-table-on-non-transactional-table-data-location" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create external table on Non-Transactional table data location.</h3>
-<p>Non-Transactional table data location will have only carbondata and carbonindex files, there will not be a metadata folder (table status and schema).
-Our SDK module currently support writing data in this format.</p>
-<p><strong>Example:</strong></p>
-<pre><code>sql(
-s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbondata' LOCATION
-|'$writerPath' """.stripMargin)
-</code></pre>
-<p>Here writer path will have carbondata and index files.
-This can be SDK output. Refer <a href="https://github.com/apache/carbondata/blob/master/docs/sdk-writer-guide.html" target=_blank>SDK Writer Guide</a>.</p>
-<p><strong>Note:</strong></p>
-<ol>
-<li>Dropping of the external table should not delete the files present in the location.</li>
-<li>When external table is created on non-transactional table data,
-external table will be registered with the schema of carbondata files.
-If multiple files with different schema is present, exception will be thrown.
-So, If table registered with one schema and files are of different schema,
-suggest to drop the external table and create again to register table with new schema.</li>
-</ol>
-<h2>
-<a id="create-database" class="anchor" href="#create-database" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CREATE DATABASE</h2>
-<p>This function creates a new database. By default the database is created in Carbon store location, but you can also specify custom location.</p>
-<pre><code>CREATE DATABASE [IF NOT EXISTS] database_name [LOCATION path];
-</code></pre>
-<h3>
-<a id="example-1" class="anchor" href="#example-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example</h3>
-<pre><code>CREATE DATABASE carbon LOCATION ?hdfs://name_cluster/dir1/carbonstore?;
-</code></pre>
-<h2>
-<a id="table-management" class="anchor" href="#table-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>TABLE MANAGEMENT</h2>
-<h3>
-<a id="show-table" class="anchor" href="#show-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW TABLE</h3>
-<p>This command can be used to list all the tables in current database or all the tables of a specific database.</p>
-<pre><code>SHOW TABLES [IN db_Name]
-</code></pre>
-<p>Example:</p>
-<pre><code>SHOW TABLES
-OR
-SHOW TABLES IN defaultdb
-</code></pre>
-<h3>
-<a id="alter-table" class="anchor" href="#alter-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>ALTER TABLE</h3>
-<p>The following section introduce the commands to modify the physical or logical state of the existing table(s).</p>
-<ul>
-<li>
-<p><strong>RENAME TABLE</strong></p>
-<p>This command is used to rename the existing table.</p>
-<pre><code>ALTER TABLE [db_name.]table_name RENAME TO new_table_name
-</code></pre>
-<p>Examples:</p>
-<pre><code>ALTER TABLE carbon RENAME TO carbonTable
-OR
-ALTER TABLE test_db.carbon RENAME TO test_db.carbonTable
-</code></pre>
-</li>
-<li>
-<p><strong>ADD COLUMNS</strong></p>
-<p>This command is used to add a new column to the existing table.</p>
-<pre><code>ALTER TABLE [db_name.]table_name ADD COLUMNS (col_name data_type,...)
-TBLPROPERTIES('DICTIONARY_INCLUDE'='col_name,...',
-'DEFAULT.VALUE.COLUMN_NAME'='default_value')
-</code></pre>
-<p>Examples:</p>
-<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING)
-</code></pre>
-<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DICTIONARY_INCLUDE'='a1')
-</code></pre>
-<pre><code>ALTER TABLE carbon ADD COLUMNS (a1 INT, b1 STRING) TBLPROPERTIES('DEFAULT.VALUE.a1'='10')
-</code></pre>
-<p>NOTE: Add Complex datatype columns is not supported.</p>
-</li>
-</ul>
-<p>Users can specify which columns to include and exclude for local dictionary generation after adding new columns. These will be appended with the already existing local dictionary include and exclude columns of main table respectively.</p>
-<pre><code>   ALTER TABLE carbon ADD COLUMNS (a1 STRING, b1 STRING) TBLPROPERTIES('LOCAL_DICTIONARY_INCLUDE'='a1','LOCAL_DICTIONARY_EXCLUDE'='b1')
-</code></pre>
-<ul>
-<li>
-<p><strong>DROP COLUMNS</strong></p>
-<p>This command is used to delete the existing column(s) in a table.</p>
-<pre><code>ALTER TABLE [db_name.]table_name DROP COLUMNS (col_name, ...)
-</code></pre>
-<p>Examples:</p>
-<pre><code>ALTER TABLE carbon DROP COLUMNS (b1)
-OR
-ALTER TABLE test_db.carbon DROP COLUMNS (b1)
-
-ALTER TABLE carbon DROP COLUMNS (c1,d1)
-</code></pre>
-<p>NOTE: Drop Complex child column is not supported.</p>
-</li>
-<li>
-<p><strong>CHANGE DATA TYPE</strong></p>
-<p>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.</p>
-<pre><code>ALTER TABLE [db_name.]table_name CHANGE col_name col_name changed_column_type
-</code></pre>
-<p>Valid Scenarios</p>
-<ul>
-<li>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.</li>
-<li>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.</li>
-<li>
-<strong>NOTE:</strong> The allowed range is 38,38 (precision, scale) and is a valid upper case scenario which is not resulting in data loss.</li>
-</ul>
-<p>Example1:Changing data type of column a1 from INT to BIGINT.</p>
-<pre><code>ALTER TABLE test_db.carbon CHANGE a1 a1 BIGINT
-</code></pre>
-<p>Example2:Changing decimal precision of column a1 from 10 to 18.</p>
-<pre><code>ALTER TABLE test_db.carbon CHANGE a1 a1 DECIMAL(18,2)
-</code></pre>
-</li>
-<li>
-<p><strong>MERGE INDEX</strong></p>
-<p>This command is used to merge all the CarbonData index files (.carbonindex) inside a segment to a single CarbonData index merge file (.carbonindexmerge). This enhances the first query performance.</p>
-<pre><code> ALTER TABLE [db_name.]table_name COMPACT 'SEGMENT_INDEX'
- ```
- 
- Examples:
- ```
- ALTER TABLE test_db.carbon COMPACT 'SEGMENT_INDEX'
- ```
- **NOTE:**
- * Merge index is not supported on streaming table.
- 
-</code></pre>
-</li>
-<li>
-<p><strong>SET and UNSET for Local Dictionary Properties</strong></p>
-<p>When set command is used, all the newly set properties will override the corresponding old properties if exists.</p>
-<p>Example to SET Local Dictionary Properties:</p>
-<pre><code>ALTER TABLE tablename SET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='false','LOCAL_DICTIONARY_THRESHOLD'='1000','LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
-</code></pre>
-<p>When Local Dictionary properties are unset, corresponding default values will be used for these properties.</p>
-<p>Example to UNSET Local Dictionary Properties:</p>
-<pre><code>ALTER TABLE tablename UNSET TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE','LOCAL_DICTIONARY_THRESHOLD','LOCAL_DICTIONARY_INCLUDE','LOCAL_DICTIONARY_EXCLUDE')
-</code></pre>
-<p><strong>NOTE:</strong> For old tables, by default, local dictionary is disabled. If user wants local dictionary for these tables, user can enable/disable local dictionary for new data at their discretion.
-This can be achieved by using the alter table set command.</p>
-</li>
-</ul>
-<h3>
-<a id="drop-table" class="anchor" href="#drop-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DROP TABLE</h3>
-<p>This command is used to delete an existing table.</p>
-<pre><code>DROP TABLE [IF EXISTS] [db_name.]table_name
-</code></pre>
-<p>Example:</p>
-<pre><code>DROP TABLE IF EXISTS productSchema.productSalesTable
-</code></pre>
-<h3>
-<a id="refresh-table" class="anchor" href="#refresh-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>REFRESH TABLE</h3>
-<p>This command is used to register Carbon table to HIVE meta store catalogue from existing Carbon table data.</p>
-<pre><code>REFRESH TABLE $db_NAME.$table_NAME
-</code></pre>
-<p>Example:</p>
-<pre><code>REFRESH TABLE dbcarbon.productSalesTable
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>The new database name and the old database name should be same.</li>
-<li>Before executing this command the old table schema and data should be copied into the new database location.</li>
-<li>If the table is aggregate table, then all the aggregate tables should be copied to the new database location.</li>
-<li>For old store, the time zone of the source and destination cluster should be same.</li>
-<li>If old cluster used HIVE meta store to store schema, refresh will not work as schema file does not exist in file system.</li>
-</ul>
-<h3>
-<a id="table-and-column-comment" class="anchor" href="#table-and-column-comment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Table and Column Comment</h3>
-<p>You can provide more information on table by using table comment. Similarly you can provide more information about a particular column using column comment.
-You can see the column comment of an existing table using describe formatted command.</p>
-<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type [COMMENT col_comment], ...)]
-  [COMMENT table_comment]
-STORED BY 'carbondata'
-[TBLPROPERTIES (property_name=property_value, ...)]
-</code></pre>
-<p>Example:</p>
-<pre><code>CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
-                              productNumber Int COMMENT 'unique serial number for product')
-COMMENT ?This is table comment?
- STORED BY 'carbondata'
- TBLPROPERTIES ('DICTIONARY_INCLUDE'='productNumber')
-</code></pre>
-<p>You can also SET and UNSET table comment using ALTER command.</p>
-<p>Example to SET table comment:</p>
-<pre><code>ALTER TABLE carbon SET TBLPROPERTIES ('comment'='this table comment is modified');
-</code></pre>
-<p>Example to UNSET table comment:</p>
-<pre><code>ALTER TABLE carbon UNSET TBLPROPERTIES ('comment');
-</code></pre>
-<h2>
-<a id="load-data" class="anchor" href="#load-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LOAD DATA</h2>
-<h3>
-<a id="load-files-to-carbondata-table" class="anchor" href="#load-files-to-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LOAD FILES TO CARBONDATA TABLE</h3>
-<p>This command is used to load csv files to carbondata, OPTIONS are not mandatory for data loading process.
-Inside OPTIONS user can provide any options like DELIMITER, QUOTECHAR, FILEHEADER, ESCAPECHAR, MULTILINE as per requirement.</p>
-<pre><code>LOAD DATA [LOCAL] INPATH 'folder_path' 
-INTO TABLE [db_name.]table_name 
-OPTIONS(property_name=property_value, ...)
-</code></pre>
-<p>You can use the following options to load data:</p>
-<ul>
-<li>
-<p><strong>DELIMITER:</strong> Delimiters can be provided in the load command.</p>
-<pre><code>OPTIONS('DELIMITER'=',')
-</code></pre>
-</li>
-<li>
-<p><strong>QUOTECHAR:</strong> Quote Characters can be provided in the load command.</p>
-<pre><code>OPTIONS('QUOTECHAR'='"')
-</code></pre>
-</li>
-<li>
-<p><strong>COMMENTCHAR:</strong> Comment Characters can be provided in the load command if user want to comment lines.</p>
-<pre><code>OPTIONS('COMMENTCHAR'='#')
-</code></pre>
-</li>
-<li>
-<p><strong>HEADER:</strong> When you load the CSV file without the file header and the file header is the same with the table schema, then add 'HEADER'='false' to load data SQL as user need not provide the file header. By default the value is 'true'.
-false: CSV file is without file header.
-true: CSV file is with file header.</p>
-<pre><code>OPTIONS('HEADER'='false') 
-</code></pre>
-<p><strong>NOTE:</strong> If the HEADER option exist and is set to 'true', then the FILEHEADER option is not required.</p>
-</li>
-<li>
-<p><strong>FILEHEADER:</strong> Headers can be provided in the LOAD DATA command if headers are missing in the source files.</p>
-<pre><code>OPTIONS('FILEHEADER'='column1,column2') 
-</code></pre>
-</li>
-<li>
-<p><strong>MULTILINE:</strong> CSV with new line character in quotes.</p>
-<pre><code>OPTIONS('MULTILINE'='true') 
-</code></pre>
-</li>
-<li>
-<p><strong>ESCAPECHAR:</strong> Escape char can be provided if user want strict validation of escape character in CSV files.</p>
-<pre><code>OPTIONS('ESCAPECHAR'='\') 
-</code></pre>
-</li>
-<li>
-<p><strong>SKIP_EMPTY_LINE:</strong> This option will ignore the empty line in the CSV file during the data load.</p>
-<pre><code>OPTIONS('SKIP_EMPTY_LINE'='TRUE/FALSE') 
-</code></pre>
-</li>
-<li>
-<p><strong>COMPLEX_DELIMITER_LEVEL_1:</strong> Split the complex type data column in a row (eg., a$b$c --&gt; Array = {a,b,c}).</p>
-<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_1'='$') 
-</code></pre>
-</li>
-<li>
-<p><strong>COMPLEX_DELIMITER_LEVEL_2:</strong> Split the complex type nested data column in a row. Applies level_1 delimiter &amp; applies level_2 based on complex data type (eg., a:b$c:d --&gt; Array&gt; = {{a,b},{c,d}}).</p>
-<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_2'=':')
-</code></pre>
-</li>
-<li>
-<p><strong>ALL_DICTIONARY_PATH:</strong> All dictionary files path.</p>
-<pre><code>OPTIONS('ALL_DICTIONARY_PATH'='/opt/alldictionary/data.dictionary')
-</code></pre>
-</li>
-<li>
-<p><strong>COLUMNDICT:</strong> Dictionary file path for specified column.</p>
-<pre><code>OPTIONS('COLUMNDICT'='column1:dictionaryFilePath1,column2:dictionaryFilePath2')
-</code></pre>
-<p><strong>NOTE:</strong> ALL_DICTIONARY_PATH and COLUMNDICT can't be used together.</p>
-</li>
-<li>
-<p><strong>DATEFORMAT/TIMESTAMPFORMAT:</strong> Date and Timestamp format for specified column.</p>
-<pre><code>OPTIONS('DATEFORMAT' = 'yyyy-MM-dd','TIMESTAMPFORMAT'='yyyy-MM-dd HH:mm:ss')
-</code></pre>
-<p><strong>NOTE:</strong> Date formats are specified by date pattern strings. The date pattern letters in CarbonData are same as in JAVA. Refer to <a href="http://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html" target=_blank rel="nofollow">SimpleDateFormat</a>.</p>
-</li>
-<li>
-<p><strong>SORT COLUMN BOUNDS:</strong> Range bounds for sort columns.</p>
-<p>Suppose the table is created with 'SORT_COLUMNS'='name,id' and the range for name is aaa<del>zzz, the value range for id is 0</del>1000. Then during data loading, we can specify the following option to enhance data loading performance.</p>
-<pre><code>OPTIONS('SORT_COLUMN_BOUNDS'='f,250;l,500;r,750')
-</code></pre>
-<p>Each bound is separated by ';' and each field value in bound is separated by ','. In the example above, we provide 3 bounds to distribute records to 4 partitions. The values 'f','l','r' can evenly distribute the records. Inside carbondata, for a record we compare the value of sort columns with that of the bounds and decide which partition the record will be forwarded to.</p>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>SORT_COLUMN_BOUNDS will be used only when the SORT_SCOPE is 'local_sort'.</li>
-<li>Carbondata will use these bounds as ranges to process data concurrently during the final sort percedure. The records will be sorted and written out inside each partition. Since the partition is sorted, all records will be sorted.</li>
-<li>Since the actual order and literal order of the dictionary column are not necessarily the same, we do not recommend you to use this feature if the first sort column is 'dictionary_include'.</li>
-<li>The option works better if your CPU usage during loading is low. If your system is already CPU tense, better not to use this option. Besides, it depends on the user to specify the bounds. If user does not know the exactly bounds to make the data distributed evenly among the bounds, loading performance will still be better than before or at least the same as before.</li>
-<li>Users can find more information about this option in the description of PR1953.</li>
-</ul>
-</li>
-<li>
-<p><strong>SINGLE_PASS:</strong> 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.</p>
-</li>
-</ul>
-<p>This option specifies whether to use single pass for loading data or not. By default this option is set to FALSE.</p>
-<pre><code> OPTIONS('SINGLE_PASS'='TRUE')
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>If this option is set to TRUE then data loading will take less time.</li>
-<li>If this option is set to some invalid value other than TRUE or FALSE then it uses the default value.</li>
-</ul>
-<p>Example:</p>
-<pre><code>LOAD DATA local inpath '/opt/rawdata/data.csv' INTO table carbontable
-options('DELIMITER'=',', 'QUOTECHAR'='"','COMMENTCHAR'='#',
-'HEADER'='false',
-'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')
-</code></pre>
-<ul>
-<li>
-<p><strong>BAD RECORDS HANDLING:</strong> Methods of handling bad records are as follows:</p>
-<ul>
-<li>Load all of the data before dealing with the errors.</li>
-<li>Clean or delete bad records before loading data or stop the loading when bad records are found.</li>
-</ul>
-<pre><code>OPTIONS('BAD_RECORDS_LOGGER_ENABLE'='true', 'BAD_RECORD_PATH'='hdfs://hacluster/tmp/carbon', 'BAD_RECORDS_ACTION'='REDIRECT', 'IS_EMPTY_DATA_BAD_RECORD'='false')
-</code></pre>
-</li>
-</ul>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>BAD_RECORDS_ACTION property can have four type of actions for bad records FORCE, REDIRECT, IGNORE and FAIL.</li>
-<li>FAIL option is its Default value. If the FAIL option is used, then data loading fails if any bad records are found.</li>
-<li>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.</li>
-<li>If the FORCE option is used, then it auto-converts the data by storing the bad records as NULL before Loading data.</li>
-<li>If the IGNORE option is used, then bad records are neither loaded nor written to the separate CSV file.</li>
-<li>In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails.</li>
-<li>The default maximum number of characters per column is 32000. If there are more than 32000 characters in a column, please refer to <em>String longer than 32000 characters</em> section.</li>
-<li>Since Bad Records Path can be specified in create, load and carbon properties.
-Therefore, value specified in load will have the highest priority, and value specified in carbon properties will have the least priority.</li>
-</ul>
-<p><strong>Bad Records Path:</strong></p>
-<p>This property is used to specify the location where bad records would be written.</p>
-<pre><code>TBLPROPERTIES('BAD_RECORDS_PATH'='/opt/badrecords'')
-</code></pre>
-<p>Example:</p>
-<pre><code>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')
-</code></pre>
-<ul>
-<li>
-<strong>GLOBAL_SORT_PARTITIONS:</strong> If the SORT_SCOPE is defined as GLOBAL_SORT, then user can specify the number of partitions to use while shuffling data for sort using GLOBAL_SORT_PARTITIONS. If it is not configured, or configured less than 1, then it uses the number of map task as reduce task. It is recommended that each reduce task deal with 512MB-1GB data.</li>
-</ul>
-<pre><code>OPTIONS('GLOBAL_SORT_PARTITIONS'='2')
-</code></pre>
-<p>NOTE:</p>
-<ul>
-<li>GLOBAL_SORT_PARTITIONS should be Integer type, the range is [1,Integer.MaxValue].</li>
-<li>It is only used when the SORT_SCOPE is GLOBAL_SORT.</li>
-</ul>
-<h3>
-<a id="insert-data-into-carbondata-table" class="anchor" href="#insert-data-into-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>INSERT DATA INTO CARBONDATA TABLE</h3>
-<p>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.</p>
-<pre><code>INSERT INTO TABLE &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
-[ WHERE { &lt;filter_condition&gt; } ]
-</code></pre>
-<p>You can also omit the <code>table</code> keyword and write your query as:</p>
-<pre><code>INSERT INTO &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
-[ WHERE { &lt;filter_condition&gt; } ]
-</code></pre>
-<p>Overwrite insert data:</p>
-<pre><code>INSERT OVERWRITE TABLE &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
-[ WHERE { &lt;filter_condition&gt; } ]
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>The source table and the CarbonData table must have the same table schema.</li>
-<li>The data type of source and destination table columns should be same</li>
-<li>INSERT INTO command does not support partial success if bad records are found, it will fail.</li>
-<li>Data cannot be loaded or updated in source table while insert from source table to target table is in progress.</li>
-</ul>
-<p>Examples</p>
-<pre><code>INSERT INTO table1 SELECT item1, sum(item2 + 1000) as result FROM table2 group by item1
-</code></pre>
-<pre><code>INSERT INTO table1 SELECT item1, item2, item3 FROM table2 where item2='xyz'
-</code></pre>
-<pre><code>INSERT OVERWRITE TABLE table1 SELECT * FROM TABLE2
-</code></pre>
-<h2>
-<a id="update-and-delete" class="anchor" href="#update-and-delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>UPDATE AND DELETE</h2>
-<h3>
-<a id="update" class="anchor" href="#update" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>UPDATE</h3>
-<p>This command will allow to update the CarbonData table based on the column expression and optional filter conditions.</p>
-<pre><code>UPDATE &lt;table_name&gt; 
-SET (column_name1, column_name2, ... column_name n) = (column1_expression , column2_expression, ... column n_expression )
-[ WHERE { &lt;filter_condition&gt; } ]
-</code></pre>
-<p>alternatively the following command can also be used for updating the CarbonData Table :</p>
-<pre><code>UPDATE &lt;table_name&gt;
-SET (column_name1, column_name2) =(select sourceColumn1, sourceColumn2 from sourceTable [ WHERE { &lt;filter_condition&gt; } ] )
-[ WHERE { &lt;filter_condition&gt; } ]
-</code></pre>
-<p><strong>NOTE:</strong> The update command fails if multiple input rows in source table are matched with single row in destination table.</p>
-<p>Examples:</p>
-<pre><code>UPDATE t3 SET (t3_salary) = (t3_salary + 9) WHERE t3_name = 'aaa1'
-</code></pre>
-<pre><code>UPDATE t3 SET (t3_date, t3_country) = ('2017-11-18', 'india') WHERE t3_salary &lt; 15003
-</code></pre>
-<pre><code>UPDATE t3 SET (t3_country, t3_name) = (SELECT t5_country, t5_name FROM t5 WHERE t5_id = 5) WHERE t3_id &lt; 5
-</code></pre>
-<pre><code>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 &lt; 5
-</code></pre>
-<pre><code>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 &gt;6
-</code></pre>
-<p>NOTE: Update Complex datatype columns is not supported.</p>
-<h3>
-<a id="delete" class="anchor" href="#delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE</h3>
-<p>This command allows us to delete records from CarbonData table.</p>
-<pre><code>DELETE FROM table_name [WHERE expression]
-</code></pre>
-<p>Examples:</p>
-<pre><code>DELETE FROM carbontable WHERE column1  = 'china'
-</code></pre>
-<pre><code>DELETE FROM carbontable WHERE column1 IN ('china', 'USA')
-</code></pre>
-<pre><code>DELETE FROM carbontable WHERE column1 IN (SELECT column11 FROM sourceTable2)
-</code></pre>
-<pre><code>DELETE FROM carbontable WHERE column1 IN (SELECT column11 FROM sourceTable2 WHERE column1 = 'USA')
-</code></pre>
-<h2>
-<a id="compaction" class="anchor" href="#compaction" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COMPACTION</h2>
-<p>Compaction improves the query performance significantly.</p>
-<p>There are several types of compaction.</p>
-<pre><code>ALTER TABLE [db_name.]table_name COMPACT 'MINOR/MAJOR/CUSTOM'
-</code></pre>
-<ul>
-<li><strong>Minor Compaction</strong></li>
-</ul>
-<p>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:</p>
-<ul>
-<li>Level 1: Merging of the segments which are not yet compacted.</li>
-<li>Level 2: Merging of the compacted segments again to form a larger segment.</li>
-</ul>
-<pre><code>ALTER TABLE table_name COMPACT 'MINOR'
-</code></pre>
-<ul>
-<li><strong>Major Compaction</strong></li>
-</ul>
-<p>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.
-Configure the property carbon.major.compaction.size with appropriate value in MB.</p>
-<p>This command merges the specified number of segments into one segment:</p>
-<pre><code>ALTER TABLE table_name COMPACT 'MAJOR'
-</code></pre>
-<ul>
-<li><strong>Custom Compaction</strong></li>
-</ul>
-<p>In Custom compaction, user can directly specify segment ids to be merged into one large segment.
-All specified segment ids should exist and be valid, otherwise compaction will fail.
-Custom compaction is usually done during the off-peak time.</p>
-<pre><code>ALTER TABLE table_name COMPACT 'CUSTOM' WHERE SEGMENT.ID IN (2,3,4)
-</code></pre>
-<p>NOTE: Compaction is unsupported for table containing Complex columns.</p>
-<ul>
-<li><strong>CLEAN SEGMENTS AFTER Compaction</strong></li>
-</ul>
-<p>Clean the segments which are compacted:</p>
-<pre><code>CLEAN FILES FOR TABLE carbon_table
-</code></pre>
-<h2>
-<a id="partition" class="anchor" href="#partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>PARTITION</h2>
-<h3>
-<a id="standard-partition" class="anchor" href="#standard-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>STANDARD PARTITION</h3>
-<p>The partition is similar as spark and hive partition, user can use any column to build partition:</p>
-<h4>
-<a id="create-partition-table" class="anchor" href="#create-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Partition Table</h4>
-<p>This command allows you to create table with partition.</p>
-<pre><code>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, ...)]
-</code></pre>
-<p>Example:</p>
-<pre><code> 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'
-</code></pre>
-<p>NOTE: Hive partition is not supported on complex datatype columns.</p>
-<h4>
-<a id="load-data-using-static-partition" class="anchor" href="#load-data-using-static-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Load Data Using Static Partition</h4>
-<p>This command allows you to load data using static partition.</p>
-<pre><code>LOAD DATA [LOCAL] INPATH 'folder_path' 
-INTO TABLE [db_name.]table_name PARTITION (partition_spec) 
-OPTIONS(property_name=property_value, ...)    
-INSERT INTO INTO TABLE [db_name.]table_name PARTITION (partition_spec) &lt;SELECT STATEMENT&gt;
-</code></pre>
-<p>Example:</p>
-<pre><code>LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.csv'
-INTO TABLE locationTable
-PARTITION (country = 'US', state = 'CA')  
-INSERT INTO TABLE locationTable
-PARTITION (country = 'US', state = 'AL')
-SELECT &lt;columns list excluding partition columns&gt; FROM another_user
-</code></pre>
-<h4>
-<a id="load-data-using-dynamic-partition" class="anchor" href="#load-data-using-dynamic-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Load Data Using Dynamic Partition</h4>
-<p>This command allows you to load data using dynamic partition. If partition spec is not specified, then the partition is considered as dynamic.</p>
-<p>Example:</p>
-<pre><code>LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.csv'
-INTO TABLE locationTable          
-INSERT INTO TABLE locationTable
-SELECT &lt;columns list excluding partition columns&gt; FROM another_user
-</code></pre>
-<h4>
-<a id="show-partitions" class="anchor" href="#show-partitions" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Show Partitions</h4>
-<p>This command gets the Hive partition information of the table</p>
-<pre><code>SHOW PARTITIONS [db_name.]table_name
-</code></pre>
-<h4>
-<a id="drop-partition" class="anchor" href="#drop-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Drop Partition</h4>
-<p>This command drops the specified Hive partition only.</p>
-<pre><code>ALTER TABLE table_name DROP [IF EXISTS] PARTITION (part_spec, ...)
-</code></pre>
-<p>Example:</p>
-<pre><code>ALTER TABLE locationTable DROP PARTITION (country = 'US');
-</code></pre>
-<h4>
-<a id="insert-overwrite" class="anchor" href="#insert-overwrite" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Insert OVERWRITE</h4>
-<p>This command allows you to insert or load overwrite on a specific partition.</p>
-<pre><code> INSERT OVERWRITE TABLE table_name
- PARTITION (column = 'partition_name')
- select_statement
-</code></pre>
-<p>Example:</p>
-<pre><code>INSERT OVERWRITE TABLE partitioned_user
-PARTITION (country = 'US')
-SELECT * FROM another_user au 
-WHERE au.country = 'US';
-</code></pre>
-<h3>
-<a id="carbondata-partitionhashrangelist----alpha-feature-this-partition-feature-does-not-support-update-and-delete-data" class="anchor" href="#carbondata-partitionhashrangelist----alpha-feature-this-partition-feature-does-not-support-update-and-delete-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CARBONDATA PARTITION(HASH,RANGE,LIST) -- Alpha feature, this partition feature does not support update and delete data.</h3>
-<p>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.</p>
-<h3>
-<a id="create-hash-partition-table" class="anchor" href="#create-hash-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Hash Partition Table</h3>
-<p>This command allows us to create hash partition.</p>
-<pre><code>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' ...)]
-</code></pre>
-<p><strong>NOTE:</strong> N is the number of hash partitions</p>
-<p>Example:</p>
-<pre><code>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')
-</code></pre>
-<h3>
-<a id="create-range-partition-table" class="anchor" href="#create-range-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create Range Partition Table</h3>
-<p>This command allows us to create range partition.</p>
-<pre><code>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, ...')]
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>The 'RANGE_INFO' must be defined in ascending order in the table properties.</li>
-<li>The default format for partition column of Date/Timestamp type is yyyy-MM-dd. Alternate formats for Date/Timestamp could be defined in CarbonProperties.</li>
-</ul>
-<p>Example:</p>
-<pre><code>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')
-</code></pre>
-<h3>
-<a id="create-list-partition-table" class="anchor" href="#create-list-partition-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create List Partition Table</h3>
-<p>This command allows us to create list partition.</p>
-<pre><code>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, ...')]
-</code></pre>
-<p><strong>NOTE:</strong> List partition supports list info in one level group.</p>
-<p>Example:</p>
-<pre><code>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')
-</code></pre>
-<h3>
-<a id="show-partitions-1" class="anchor" href="#show-partitions-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Show Partitions</h3>
-<p>The following command is executed to get the partition information of the table</p>
-<pre><code>SHOW PARTITIONS [db_name.]table_name
-</code></pre>
-<h3>
-<a id="add-a-new-partition" class="anchor" href="#add-a-new-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Add a new partition</h3>
-<pre><code>ALTER TABLE [db_name].table_name ADD PARTITION('new_partition')
-</code></pre>
-<h3>
-<a id="split-a-partition" class="anchor" href="#split-a-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Split a partition</h3>
-<pre><code>ALTER TABLE [db_name].table_name SPLIT PARTITION(partition_id) INTO('new_partition1', 'new_partition2'...)
-</code></pre>
-<h3>
-<a id="drop-a-partition" class="anchor" href="#drop-a-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Drop a partition</h3>
-<p>Only drop partition definition, but keep data</p>
-<pre><code>  ALTER TABLE [db_name].table_name DROP PARTITION(partition_id)
-</code></pre>
-<p>Drop both partition definition and data</p>
-<pre><code>ALTER TABLE [db_name].table_name DROP PARTITION(partition_id) WITH DATA
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>Hash partition table is not supported for ADD, SPLIT and DROP commands.</li>
-<li>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.</li>
-</ul>
-<pre><code>SegmentDir/0_batchno0-0-1502703086921.carbonindex
-          ^
-SegmentDir/part-0-0_batchno0-0-1502703086921.carbondata
-                   ^
-</code></pre>
-<p>Here are some useful tips to improve query performance of carbonData partition table:</p>
-<ul>
-<li>The partitioned column can be excluded from SORT_COLUMNS, this will let other columns to do the efficient sorting.</li>
-<li>When writing SQL on a partition table, try to use filters on the partition column.</li>
-</ul>
-<h2>
-<a id="bucketing" class="anchor" href="#bucketing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>BUCKETING</h2>
-<p>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.</p>
-<pre><code>CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
-                  [(col_name data_type, ...)]
-STORED BY 'carbondata'
-TBLPROPERTIES('BUCKETNUMBER'='noOfBuckets',
-'BUCKETCOLUMNS'='columnname')
-</code></pre>
-<p><strong>NOTE:</strong></p>
-<ul>
-<li>Bucketing cannot be performed for columns of Complex Data Types.</li>
-<li>Columns in the BUCKETCOLUMN parameter must be dimensions. The BUCKETCOLUMN parameter cannot be a measure or a combination of measures and dimensions.</li>
-</ul>
-<p>Example:</p>
-<pre><code>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')
-</code></pre>
-<h2>
-<a id="segment-management" class="anchor" href="#segment-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SEGMENT MANAGEMENT</h2>
-<h3>
-<a id="show-segment" class="anchor" href="#show-segment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW SEGMENT</h3>
-<p>This command is used to list the segments of CarbonData table.</p>
-<pre><code>SHOW [HISTORY] SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
-</code></pre>
-<p>Example:
-Show visible segments</p>
-<pre><code>SHOW SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4
-</code></pre>
-<p>Show all segments, include invisible segments</p>
-<pre><code>SHOW HISTORY SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4
-</code></pre>
-<h3>
-<a id="delete-segment-by-id" class="anchor" href="#delete-segment-by-id" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY ID</h3>
-<p>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.</p>
-<p>The following command will get the segmentID.</p>
-<pre><code>SHOW SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
-</code></pre>
-<p>After you retrieve the segment ID of the segment that you want to delete, execute the following command to delete the selected segment.</p>
-<pre><code>DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.ID IN (segment_id1, segments_id2, ...)
-</code></pre>
-<p>Example:</p>
-<pre><code>DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0)
-DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.ID IN (0,5,8)
-</code></pre>
-<h3>
-<a id="delete-segment-by-date" class="anchor" href="#delete-segment-by-date" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE SEGMENT BY DATE</h3>
-<p>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.</p>
-<pre><code>DELETE FROM TABLE [db_name.]table_name WHERE SEGMENT.STARTTIME BEFORE DATE_VALUE
-</code></pre>
-<p>Example:</p>
-<pre><code>DELETE FROM TABLE CarbonDatabase.CarbonTable WHERE SEGMENT.STARTTIME BEFORE '2017-06-01 12:05:06' 
-</code></pre>
-<h3>
-<a id="query-data-with-specified-segments" class="anchor" href="#query-data-with-specified-segments" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>QUERY DATA WITH SPECIFIED SEGMENTS</h3>
-<p>This command is used to read data from specified segments during CarbonScan.</p>
-<p>Get the Segment ID:</p>
-<pre><code>SHOW SEGMENTS FOR TABLE [db_name.]table_name LIMIT number_of_segments
-</code></pre>
-<p>Set the segment IDs for table</p>
-<pre><code>SET carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt; = &lt;list of segment IDs&gt;
-</code></pre>
-<p><strong>NOTE:</strong>
-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.</p>
-<p>If user wants to query with segments reading in multi threading mode, then CarbonSession. threadSet can be used instead of SET query.</p>
-<pre><code>CarbonSession.threadSet ("carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt;","&lt;list of segment IDs&gt;");
-</code></pre>
-<p>Reset the segment IDs</p>
-<pre><code>SET carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt; = *;
-</code></pre>
-<p>If user wants to query with segments reading in multi threading mode, then CarbonSession. threadSet can be used instead of SET query.</p>
-<pre><code>CarbonSession.threadSet ("carbon.input.segments.&lt;database_name&gt;.&lt;table_name&gt;","*");
-</code></pre>
-<p><strong>Examples:</strong></p>
-<ul>
-<li>Example to show the list of segment IDs,segment status, and other required details and then specify the list of segments to be read.</li>
-</ul>
-<pre><code>SHOW SEGMENTS FOR carbontable1;
-
-SET carbon.input.segments.db.carbontable1 = 1,3,9;
-</code></pre>
-<ul>
-<li>Example to query with segments reading in multi threading mode:</li>
-</ul>
-<pre><code>CarbonSession.threadSet ("carbon.input.segments.db.carbontable_Multi_Thread","1,3");
-</code></pre>
-<ul>
-<li>Example for threadset in multithread environment (following shows how it is used in Scala code):</li>
-</ul>
-<pre><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();
-   }
- }
-</code></pre>
-</div>
-</div>
-</div>
-</div>
-<div class="doc-footer">
-    <a href="#top" class="scroll-top">Top</a>
-</div>
-</div>
-</section>
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>


[20/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/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
index 7ac5a3f..37c398c 100644
--- a/src/site/markdown/quick-start-guide.md
+++ b/src/site/markdown/quick-start-guide.md
@@ -19,9 +19,9 @@
 This tutorial provides a quick introduction to using CarbonData.To follow along with this guide, first download a packaged release of CarbonData from the [CarbonData website](https://dist.apache.org/repos/dist/release/carbondata/).Alternatively it can be created following [Building CarbonData](https://github.com/apache/carbondata/tree/master/build) steps.
 
 ##  Prerequisites
-* Spark 2.2.1 version is installed and running.CarbonData supports Spark versions upto 2.2.1.Please follow steps described in [Spark docs website](https://spark.apache.org/docs/latest) for installing and running Spark.
+* CarbonData supports Spark versions upto 2.2.1.Please download Spark package from [Spark website](https://spark.apache.org/downloads.html)
 
-* Create a sample.csv file using the following commands. The CSV file is required for loading data into CarbonData.
+* Create a sample.csv file using the following commands. The CSV file is required for loading data into CarbonData
 
   ```
   cd carbondata
@@ -33,7 +33,7 @@ This tutorial provides a quick introduction to using CarbonData.To follow along
   EOF
   ```
 
-## Deployment modes
+## Integration
 
 CarbonData can be integrated with Spark and Presto Execution Engines.The below documentation guides on Installing and Configuring with these execution engines.
 
@@ -45,16 +45,13 @@ CarbonData can be integrated with Spark and Presto Execution Engines.The below d
 
 [Installing and Configuring CarbonData on Spark on YARN Cluster](#installing-and-configuring-carbondata-on-spark-on-yarn-cluster)
 
+[Installing and Configuring CarbonData Thrift Server for Query Execution](#query-execution-using-carbondata-thrift-server)
+
 
 ### Presto
 [Installing and Configuring CarbonData on Presto](#installing-and-configuring-carbondata-on-presto)
 
 
-## Querying Data
-
-[Query Execution using CarbonData Thrift Server](#query-execution-using-carbondata-thrift-server)
-
-## 
 
 ## Installing and Configuring CarbonData to run locally with Spark Shell
 
@@ -95,12 +92,12 @@ val carbon = SparkSession.builder().config(sc.getConf)
 
 ```
 scala>carbon.sql("CREATE TABLE
-                        IF NOT EXISTS test_table(
-                                  id string,
-                                  name string,
-                                  city string,
-                                  age Int)
-                       STORED BY 'carbondata'")
+                    IF NOT EXISTS test_table(
+                    id string,
+                    name string,
+                    city string,
+                    age Int)
+                  STORED AS carbondata")
 ```
 
 ###### Loading Data to a Table
@@ -296,8 +293,12 @@ hdfs://<host_name>:port/user/hive/warehouse/carbon.store
 
 ## Installing and Configuring CarbonData on Presto
 
+**NOTE:** **CarbonData tables cannot be created nor loaded from Presto.User need to create CarbonData Table and load data into it
+either with [Spark](#installing-and-configuring-carbondata-to-run-locally-with-spark-shell) or [SDK](./sdk-guide.md).
+Once the table is created,it can be queried from Presto.**
 
-* ### Installing Presto
+
+### Installing Presto
 
  1. Download the 0.187 version of Presto using:
     `wget https://repo1.maven.org/maven2/com/facebook/presto/presto-server/0.187/presto-server-0.187.tar.gz`
@@ -429,9 +430,29 @@ select * from system.runtime.nodes;
 ```
 Now you can use the Presto CLI on the coordinator to query data sources in the catalog using the Presto workers.
 
+List the schemas(databases) available
+
+```
+show schemas;
+```
+
+Selected the schema where CarbonData table resides
+
+```
+use carbonschema;
+```
+
+List the available tables
+
+```
+show tables;
+```
+
+Query from the available tables
+
+```
+select * from carbon_table;
+```
+
 **Note :** Create Tables and data loads should be done before executing queries as we can not create carbon table from this interface.
 
-<script>
-// Show selected style on nav item
-$(function() { $('.b-nav__quickstart').addClass('selected'); });
-</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/release-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/release-guide.md b/src/site/markdown/release-guide.md
index 40a9058..e626ccb 100644
--- a/src/site/markdown/release-guide.md
+++ b/src/site/markdown/release-guide.md
@@ -420,9 +420,3 @@ _Checklist to declare the process completed:_
 1. Release announced on the user@ mailing list.
 2. Release announced on the Incubator's general@ mailing list.
 3. Completion declared on the dev@ mailing list.
-
-
-<script>
-// Show selected style on nav item
-$(function() { $('.b-nav__release').addClass('selected'); });
-</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/s3-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/s3-guide.md b/src/site/markdown/s3-guide.md
index 37f157c..a2e5f07 100644
--- a/src/site/markdown/s3-guide.md
+++ b/src/site/markdown/s3-guide.md
@@ -88,7 +88,3 @@ recommended to set the configurable lock path property([carbon.lock.path](./conf
  to a HDFS directory.
 2. Concurrent data manipulation operations are not supported. Object stores follow eventual consistency semantics, i.e., any put request might take some time to reflect when trying to list. This behaviour causes the data read is always not consistent or not the latest.
 
-<script>
-// Show selected style on nav item
-$(function() { $('.b-nav__s3').addClass('selected'); });
-</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/sdk-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/sdk-guide.md b/src/site/markdown/sdk-guide.md
index 66f3d61..d786406 100644
--- a/src/site/markdown/sdk-guide.md
+++ b/src/site/markdown/sdk-guide.md
@@ -7,7 +7,7 @@
     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.
@@ -16,8 +16,16 @@
 -->
 
 # SDK Guide
-In the carbon jars package, there exist a carbondata-store-sdk-x.x.x-SNAPSHOT.jar, including SDK writer and reader.
+
+CarbonData provides SDK to facilitate
+
+1. [Writing carbondata files from other application which does not use Spark](#sdk-writer)
+2. [Reading carbondata files from other application which does not use Spark](#sdk-reader)
+
 # SDK Writer
+
+In the carbon jars package, there exist a carbondata-store-sdk-x.x.x-SNAPSHOT.jar, including SDK writer and reader.
+
 This SDK writer, writes carbondata file and carbonindex file at a given path.
 External client can make use of this writer to convert other format data or live data to create carbondata and index files.
 These SDK writer output contains just a carbondata and carbonindex files. No metadata folder will be present.
@@ -867,8 +875,3 @@ public String getProperty(String key, String defaultValue);
 ```
 Reference : [list of carbon properties](./configuration-parameters.md)
 
-
-<script>
-// Show selected style on nav item
-$(function() { $('.b-nav__api').addClass('selected'); });
-</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/segment-management-on-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/segment-management-on-carbondata.md b/src/site/markdown/segment-management-on-carbondata.md
index a519c88..fe0cbd4 100644
--- a/src/site/markdown/segment-management-on-carbondata.md
+++ b/src/site/markdown/segment-management-on-carbondata.md
@@ -140,15 +140,3 @@ concept which helps to maintain consistency of data and easy transaction managem
      }
    }
   ```
-
-
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__docs').addClass('selected');
-  // Display docs subnav items
-  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
-    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/streaming-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/streaming-guide.md b/src/site/markdown/streaming-guide.md
index 2f8aa5e..3b71662 100644
--- a/src/site/markdown/streaming-guide.md
+++ b/src/site/markdown/streaming-guide.md
@@ -7,7 +7,7 @@
     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.
@@ -17,6 +17,24 @@
 
 # CarbonData Streaming Ingestion
 
+- [Streaming Table Management](#quick-example)
+  - [Create table with streaming property](#create-table-with-streaming-property)
+  - [Alter streaming property](#alter-streaming-property)
+  - [Acquire streaming lock](#acquire-streaming-lock)
+  - [Create streaming segment](#create-streaming-segment)
+  - [Change Stream segment status](#change-segment-status)
+  - [Handoff "streaming finish" segment to columnar segment](#handoff-streaming-finish-segment-to-columnar-segment)
+  - [Auto handoff streaming segment](#auto-handoff-streaming-segment)
+  - [Stream data parser](#stream-data-parser)
+  - [Close streaming table](#close-streaming-table)
+  - [Constraints](#constraint)
+- [StreamSQL](#streamsql)
+  - [Defining Streaming Table](#streaming-table)
+  - [Streaming Job Management](#streaming-job-management)
+    - [START STREAM](#start-stream)
+    - [STOP STREAM](#stop-stream)
+    - [SHOW STREAMS](#show-streams)
+
 ## Quick example
 Download and unzip spark-2.2.0-bin-hadoop2.7.tgz, and export $SPARK_HOME
 
@@ -68,7 +86,7 @@ Start spark-shell in new terminal, type :paste, then copy and run the following
       | col1 INT,
       | col2 STRING
       | )
-      | STORED BY 'carbondata'
+      | STORED AS carbondata
       | TBLPROPERTIES('streaming'='true')""".stripMargin)
 
  val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "carbon_table")(spark)
@@ -116,19 +134,19 @@ streaming table using following DDL.
   col1 INT,
   col2 STRING
  )
- STORED BY 'carbondata'
+ STORED AS carbondata
  TBLPROPERTIES('streaming'='true')
 ```
 
  property name | default | description
  ---|---|--- 
  streaming | false |Whether to enable streaming ingest feature for this table <br /> Value range: true, false 
- 
+
  "DESC FORMATTED" command will show streaming property.
  ```sql
  DESC FORMATTED streaming_table
  ```
- 
+
 ## Alter streaming property
 For an old table, use ALTER TABLE command to set the streaming property.
 ```sql
@@ -261,14 +279,145 @@ ALTER TABLE streaming_table COMPACT 'close_streaming'
 7. block drop the streaming table while the streaming ingestion is running.
 
 
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__docs').addClass('selected');
 
-  // Display docs subnav items
-  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
-    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>
+## StreamSQL
+
+
+
+### Streaming Table
+
+**Example**
+
+Following example shows how to start a streaming ingest job
+
+```
+    sql(
+      s"""
+         |CREATE TABLE source(
+         | id INT,
+         | name STRING,
+         | city STRING,
+         | salary FLOAT,
+         | tax DECIMAL(8,2),
+         | percent double,
+         | birthday DATE,
+         | register TIMESTAMP,
+         | updated TIMESTAMP
+         |)
+         |STORED BY carbondata
+         |TBLPROPERTIES (
+         | 'format'='csv',
+         | 'path'='$csvDataDir'
+         |)
+      """.stripMargin)
+
+    sql(
+      s"""
+         |CREATE TABLE sink(
+         | id INT,
+         | name STRING,
+         | city STRING,
+         | salary FLOAT,
+         | tax DECIMAL(8,2),
+         | percent double,
+         | birthday DATE,
+         | register TIMESTAMP,
+         | updated TIMESTAMP
+         |)
+         |STORED BY carbondata
+         |TBLPROPERTIES (
+         |  'streaming'='true'
+         |)
+      """.stripMargin)
+
+    sql(
+      """
+        |START STREAM job123 ON TABLE sink
+        |STMPROPERTIES(
+        |  'trigger'='ProcessingTime',
+        |  'interval'='1 seconds')
+        |AS
+        |  SELECT *
+        |  FROM source
+        |  WHERE id % 2 = 1
+      """.stripMargin)
+
+    sql("STOP STREAM job123")
+
+    sql("SHOW STREAMS [ON TABLE tableName]")
+```
+
+
+
+In above example, two table is created: source and sink. The `source` table's format is `csv` and `sink` table format is `carbon`. Then a streaming job is created to stream data from source table to sink table.
+
+These two tables are normal carbon table, they can be queried independently.
+
+
+
+### Streaming Job Management
+
+As above example shown:
+
+- `START STREAM jobName ON TABLE tableName` is used to start a streaming ingest job. 
+- `STOP STREAM jobName` is used to stop a streaming job by its name
+- `SHOW STREAMS [ON TABLE tableName]` is used to print streaming job information
+
+
+
+##### START STREAM
+
+When this is issued, carbon will start a structured streaming job to do the streaming ingestion. Before launching the job, system will validate:
+
+- The format of table specified in CTAS FROM clause must be one of: csv, json, text, parquet, kafka, socket.  These are formats supported by spark 2.2.0 structured streaming
+
+- User should pass the options of the streaming source table in its TBLPROPERTIES when creating it. StreamSQL will pass them transparently to spark when creating the streaming job. For example:
+
+  ```SQL
+  CREATE TABLE source(
+    name STRING,
+    age INT
+  )
+  STORED BY carbondata
+  TBLPROPERTIES(
+    'format'='socket',
+    'host'='localhost',
+    'port'='8888'
+  )
+  ```
+
+  will translate to
+
+  ```Scala
+  spark.readStream
+  	 .schema(tableSchema)
+  	 .format("socket")
+  	 .option("host", "localhost")
+  	 .option("port", "8888")
+  ```
+
+
+
+- The sink table should have a TBLPROPERTY `'streaming'` equal to `true`, indicating it is a streaming table.
+- In the given STMPROPERTIES, user must specify `'trigger'`, its value must be `ProcessingTime` (In future, other value will be supported). User should also specify interval value for the streaming job.
+- If the schema specifid in sink table is different from CTAS, the streaming job will fail
+
+
+
+##### STOP STREAM
+
+When this is issued, the streaming job will be stopped immediately. It will fail if the jobName specified is not exist.
+
+
+
+##### SHOW STREAMS
+
+`SHOW STREAMS ON TABLE tableName` command will print the streaming job information as following
+
+| Job name | status  | Source | Sink | start time          | time elapsed |
+| -------- | ------- | ------ | ---- | ------------------- | ------------ |
+| job123   | Started | device | fact | 2018-02-03 14:32:42 | 10d2h32m     |
+
+`SHOW STREAMS` command will show all stream jobs in the system.
+
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/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
index 35e41ba..fee80f6 100644
--- a/src/site/markdown/supported-data-types-in-carbondata.md
+++ b/src/site/markdown/supported-data-types-in-carbondata.md
@@ -46,15 +46,4 @@
 
   * Other Types
     * BOOLEAN
-    
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__docs').addClass('selected');
-
-  // Display docs subnav items
-  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
-    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/timeseries-datamap-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/timeseries-datamap-guide.md b/src/site/markdown/timeseries-datamap-guide.md
index d3ef3c6..3f849c4 100644
--- a/src/site/markdown/timeseries-datamap-guide.md
+++ b/src/site/markdown/timeseries-datamap-guide.md
@@ -17,9 +17,9 @@
 
 # CarbonData Timeseries DataMap
 
-* [Timeseries DataMap Introduction](#timeseries-datamap-intoduction)
-* [Compaction](#compacting-pre-aggregate-tables)
-* [Data Management](#data-management-with-pre-aggregate-tables)
+* [Timeseries DataMap Introduction](#timeseries-datamap-introduction-alpha-feature)
+* [Compaction](#compacting-timeseries-datamp)
+* [Data Management](#data-management-on-timeseries-datamap)
 
 ## Timeseries DataMap Introduction (Alpha Feature)
 Timeseries DataMap is a pre-aggregate table implementation based on 'pre-aggregate' DataMap.
@@ -153,14 +153,3 @@ Same applies to timeseries datamap.
 Refer to Data Management section in [preaggregation datamap](./preaggregate-datamap-guide.md).
 Same applies to timeseries datamap.
 
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__datamap').addClass('selected');
-  
-  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
-    // Display datamap subnav items
-    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/usecases.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/usecases.md b/src/site/markdown/usecases.md
new file mode 100644
index 0000000..277c455
--- /dev/null
+++ b/src/site/markdown/usecases.md
@@ -0,0 +1,215 @@
+# Use Cases
+
+CarbonData is useful in various analytical work loads.Some of the most typical usecases where CarbonData is being used is documented here.
+
+CarbonData is used for but not limited to
+
+- ### Bank
+
+  - fraud detection analysis
+  - risk profile analysis
+  - As a zip table to update the daily balance of customers
+
+- ### Telecom
+
+  - Detection of signal anamolies for VIP customers for providing improved customer experience
+  - Analysis of MR,CHR records of GSM data to determine the tower load at a particular time period and rebalance the tower configuration
+  - Analysis of access sites, video, screen size, streaming bandwidth, quality to determine the network quality,routing configuration
+
+- ### Web/Internet
+
+  - Analysis of page or video being accessed,server loads, streaming quality, screen size
+
+- ### Smart City
+
+  - Vehicle tracking analysis
+  - Unusual behaviour analysis
+
+
+
+These use cases can be broadly classified into below categories:
+
+- Full scan/Detailed/Interactive queries
+- Aggregation/OLAP BI queries
+- Real time Ingestion(Streaming) and queries
+
+
+
+## Detailed Queries in the Telecom scenario
+
+### Scenario
+
+User wants to analyse all the CHR(Call History Record) and MR(Measurement Records) of the mobile subscribers in order to identify the service failures within 10 secs.Also user wants to run machine learning models on the data to fairly estimate the reasons and time of probable failures and take action ahead to meet the SLA(Service Level Agreements) of VIP customers. 
+
+### Challenges
+
+- Data incoming rate might vary based on the user concentration at a particular period of time.Hence higher data load speeds are required
+- Cluster needs to be well utilised and share the cluster among various applications for better resource consumption and savings
+- Queries needs to be interactive.ie., the queries fetch small data and need to be returned in seconds
+- Data Loaded into the system every few minutes.
+
+### Solution
+
+Setup a Hadoop + Spark + CarbonData cluster managed by YARN.
+
+Proposed the following configurations for CarbonData.(These tunings were proposed before CarbonData introduced SORT_COLUMNS parameter using which the sort order and schema order could be different.)
+
+Add the frequently used columns to the left of the table definition.Add it in the increasing order of cardinality.It was suggested to keep msisdn,imsi columns in the beginning of the schema.With latest CarbonData, SORT_COLUMNS needs to be configured msisdn,imsi in the beginning.
+
+Add timestamp column to the right of the schema as it is naturally increasing.
+
+Create two separate YARN queues for Query and Data Loading.
+
+Apart from these, the following CarbonData configuration was suggested to be configured in the cluster.
+
+
+
+| Configuration for | Parameter                               | Value  | Description |
+|------------------ | --------------------------------------- | ------ | ----------- |
+| Data Loading | carbon.graph.rowset.size                | 100000 | Based on the size of each row, this determines the memory required during data loading.Higher value leads to increased memory foot print |
+| Data Loading | carbon.number.of.cores.while.loading    | 12     | More cores can improve data loading speed |
+| Data Loading | carbon.sort.size                        | 100000 | Number of records to sort at a time.More number of records configured will lead to increased memory foot print |
+| Data Loading | table_blocksize                         | 256  | To efficiently schedule multiple tasks during query |
+| Data Loading | carbon.sort.intermediate.files.limit    | 100    | Increased to 100 as number of cores are more.Can perform merging in backgorund.If less number of files to merge, sort threads would be idle |
+| Data Loading | carbon.use.local.dir                    | TRUE   | yarn application directory will be usually on a single disk.YARN would be configured with multiple disks to be used as temp or to assign randomly to applications.Using the yarn temp directory will allow carbon to use multiple disks and improve IO performance |
+| Data Loading | carbon.use.multiple.temp.dir            | TRUE   | multiple disks to write sort files will lead to better IO and reduce the IO bottleneck |
+| Compaction | carbon.compaction.level.threshold       | 6,6    | Since frequent small loads, compacting more segments will give better query results |
+| Compaction | carbon.enable.auto.load.merge           | true   | Since data loading is small,auto compacting keeps the number of segments less and also compaction can complete in  time |
+| Compaction | carbon.number.of.cores.while.compacting | 4      | Higher number of cores can improve the compaction speed |
+| Compaction | carbon.major.compaction.size            | 921600 | Sum of several loads to combine into single segment |
+
+
+
+### Results Achieved
+
+| Parameter                                 | Results          |
+| ----------------------------------------- | ---------------- |
+| Query                                     | < 3 Sec          |
+| Data Loading Speed                        | 40 MB/s Per Node |
+| Concurrent query performance (20 queries) | < 10 Sec         |
+
+
+
+## Detailed Queries in the Smart City scenario
+
+### Scenario
+
+User wants to analyse the person/vehicle movement and behavior during a certain time period.This output data needs to be joined with a external table for Human details extraction.The query will be run with different time period as filter to identify potential behavior mismatch.
+
+### Challenges
+
+Data generated per day is very huge.Data needs to be loaded multiple times per day to accomodate the incoming data size.
+
+Data Loading done once in 6 hours.
+
+### Solution
+
+Setup a Hadoop + Spark + CarbonData cluster managed by YARN.
+
+Since data needs to be queried for a time period, it was recommended to keep the time column at the beginning of schema.
+
+Use table block size as 512MB.
+
+Use local sort mode.
+
+Apart from these, the following CarbonData configuration was suggested to be configured in the cluster.
+
+Use all columns are no-dictionary as the cardinality is high.
+
+| Configuration for | Parameter                               | Value                   | Description |
+| ------------------| --------------------------------------- | ----------------------- | ------------------|
+| Data Loading | carbon.graph.rowset.size                | 100000                  | Based on the size of each row, this determines the memory required during data loading.Higher value leads to increased memory foot print |
+| Data Loading | enable.unsafe.sort                      | TRUE                    | Temporary data generated during sort is huge which causes GC bottlenecks.Using unsafe reduces the pressure on GC |
+| Data Loading | enable.offheap.sort                     | TRUE                    | Temporary data generated during sort is huge which causes GC bottlenecks.Using offheap reduces the pressure on GC.offheap can be accessed through java unsafe.hence enable.unsafe.sort needs to be true |
+| Data Loading | offheap.sort.chunk.size.in.mb           | 128                     | Size of memory to allocate for sorting.Can increase this based on the memory available |
+| Data Loading | carbon.number.of.cores.while.loading    | 12                      | Higher cores can improve data loading speed |
+| Data Loading | carbon.sort.size                        | 100000                  | Number of records to sort at a time.More number of records configured will lead to increased memory foot print |
+| Data Loading | table_blocksize                         | 512                     | To efficiently schedule multiple tasks during query.This size depends on data scenario.If data is such that the filters would select less number of blocklets to scan, keeping higher number works well.If the number blocklets to scan is more, better to reduce the size as more tasks can be scheduled in parallel. |
+| Data Loading | carbon.sort.intermediate.files.limit    | 100                     | Increased to 100 as number of cores are more.Can perform merging in backgorund.If less number of files to merge, sort threads would be idle |
+| Data Loading | carbon.use.local.dir                    | TRUE                    | yarn application directory will be usually on a single disk.YARN would be configured with multiple disks to be used as temp or to assign randomly to applications.Using the yarn temp directory will allow carbon to use multiple disks and improve IO performance |
+| Data Loading | carbon.use.multiple.temp.dir            | TRUE                    | multiple disks to write sort files will lead to better IO and reduce the IO bottleneck |
+| Data Loading | sort.inmemory.size.in.mb                | 92160 | Memory allocated to do inmemory sorting.When more memory is available in the node, configuring this will retain more sort blocks in memory so that the merge sort is faster due to no/very less IO |
+| Compaction | carbon.major.compaction.size            | 921600                  | Sum of several loads to combine into single segment |
+| Compaction | carbon.number.of.cores.while.compacting | 12                      | Higher number of cores can improve the compaction speed.Data size is huge.Compaction need to use more threads to speed up the process |
+| Compaction | carbon.enable.auto.load.merge           | FALSE                   | Doing auto minor compaction is costly process as data size is huge.Perform manual compaction when  the cluster is less loaded |
+| Query | carbon.enable.vector.reader             | true                    | To fetch results faster, supporting spark vector processing will speed up the query |
+| Query | enable.unsafe.in.query.procressing      | true                    | Data that needs to be scanned in huge which in turn generates more short lived Java objects.This cause pressure of GC.using unsafe and offheap will reduce the GC overhead |
+| Query | use.offheap.in.query.processing         | true                    | Data that needs to be scanned in huge which in turn generates more short lived Java objects.This cause pressure of GC.using unsafe and offheap will reduce the GC overhead.offheap can be accessed through java unsafe.hence enable.unsafe.in.query.procressing needs to be true |
+| Query | enable.unsafe.columnpage                | TRUE                    | Keep the column pages in offheap memory so that the memory overhead due to java object is less and also reduces GC pressure. |
+| Query | carbon.unsafe.working.memory.in.mb      | 10240                   | Amount of memory to use for offheap operations.Can increase this memory based on the data size |
+
+
+
+### Results Achieved
+
+| Parameter                              | Results          |
+| -------------------------------------- | ---------------- |
+| Query (Time Period spanning 1 segment) | < 10 Sec         |
+| Data Loading Speed                     | 45 MB/s Per Node |
+
+
+
+## OLAP/BI Queries in the web/Internet scenario
+
+### Scenario
+
+An Internet company wants to analyze the average download speed, kind of handsets used in a particular region/area,kind of Apps being used, what kind of videos are trending in a particular region to enable them to identify the appropriate resolution size of videos to speed up transfer, and perform many more analysis to serve th customers better.
+
+### Challenges
+
+Since data is being queried by a BI tool, all the queries contain group by, which means CarbonData need to return more records as limit cannot be pushed down to carbondata layer.
+
+Results have to be returned faster as the BI tool would not respond till the data is fetched, causing bad user experience.
+
+Data might be loaded less frequently(once or twice in a day), but raw data size is huge, which causes the group by queries to run slower.
+
+Concurrent queries can be more due to the BI dashboard
+
+### Goal
+
+1. Aggregation queries are faster
+2. Concurrency is high(Number of concurrent queries supported)
+
+### Solution
+
+- Use table block size as 128MB so that pruning is more effective
+- Use global sort mode so that the data to be fetched are grouped together
+- Create pre-aggregate tables for non timestamp based group by queries
+- For queries containing group by date, create timeseries based Datamap(pre-aggregate) tables so that the data is rolled up during creation and fetch is faster
+- Reduce the Spark shuffle partitions.(In our configuration on 14 node cluster, it was reduced to 35 from default of 200)
+- Enable global dictionary for columns which have less cardinalities.Aggregation can be done on encoded data, there by improving the performance
+- For columns whose cardinality is high,enable the local dictionary so that store size is less and can take dictionary benefit for scan
+
+## Handling near realtime data ingestion scenario
+
+### Scenario
+
+Need to support storing of continously arriving data and make it available immediately for query.
+
+### Challenges
+
+When the data ingestion is near real time and the data needs to be available for query immediately, usual scenario is to do data loading in micro batches.But this causes the problem of generating many small files.This poses two problems:
+
+1. Small file handling in HDFS is inefficient
+2. CarbonData will suffer in query performance as all the small files will have to be queried when filter is on non time column
+
+CarbonData will suffer in query performance as all the small files will have to be queried when filter is on non time column.
+
+Since data is continouly arriving, allocating resources for compaction might not be feasible.
+
+### Goal
+
+1. Data is available in near real time for query as it arrives
+2. CarbonData doesnt suffer from small files problem
+
+### Solution
+
+- Use Streaming tables support of CarbonData
+- Configure the carbon.streaming.segment.max.size property to higher value(default is 1GB) if a bit slower query performance is not a concern
+- Configure carbon.streaming.auto.handoff.enabled to true so that after the  carbon.streaming.segment.max.size is reached, the segment is converted into format optimized for query
+- Disable auto compaction.Manually trigger the minor compaction with default 4,3 when the cluster is not busy
+- Manually trigger Major compaction based on the size of segments and the frequency with which the segments are being created
+- Enable local dictionary
+
+
+


[09/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/how-to-contribute-to-apache-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/how-to-contribute-to-apache-carbondata.html b/src/main/webapp/how-to-contribute-to-apache-carbondata.html
new file mode 100644
index 0000000..946013f
--- /dev/null
+++ b/src/main/webapp/how-to-contribute-to-apache-carbondata.html
@@ -0,0 +1,371 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="how-to-contribute-to-apache-carbondata" class="anchor" href="#how-to-contribute-to-apache-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>How to contribute to Apache CarbonData</h1>
+<p>The Apache CarbonData community welcomes all kinds of contributions from anyone with a passion for
+faster data format! Apache CarbonData is a new file format for faster interactive query using
+advanced columnar storage, index, compression and encoding techniques to improve computing
+efficiency,in turn it will help speedup queries an order of magnitude faster over PetaBytes of data.</p>
+<p>We use a review-then-commit workflow in CarbonData for all contributions.</p>
+<ul>
+<li>Engage -&gt; Design -&gt; Code -&gt; Review -&gt; Commit</li>
+</ul>
+<h2>
+<a id="engage" class="anchor" href="#engage" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Engage</h2>
+<h3>
+<a id="mailing-lists" class="anchor" href="#mailing-lists" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Mailing list(s)</h3>
+<p>We discuss design and implementation issues on <a href="mailto:dev@carbondata.apache.org">dev@carbondata.apache.org</a> Join by
+emailing <a href="mailto:dev-subscribe@carbondata.apache.org">dev-subscribe@carbondata.apache.org</a></p>
+<h3>
+<a id="apache-jira" class="anchor" href="#apache-jira" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Apache JIRA</h3>
+<p>We use <a href="https://issues.apache.org/jira/browse/CARBONDATA" target=_blank rel="nofollow">Apache JIRA</a> as an issue tracking and
+project management tool, as well as a way to communicate among a very diverse and distributed set
+of contributors. To be able to gather feedback, avoid frustration, and avoid duplicated efforts all
+CarbonData-related work should be tracked there.</p>
+<p>If you do not already have an Apache JIRA account, sign up <a href="https://issues.apache.org/jira/" target=_blank rel="nofollow">here</a>.</p>
+<p>If a quick search doesn?t turn up an existing JIRA issue for the work you want to contribute,
+create it. Please discuss your proposal with a committer or the component lead in JIRA or,
+alternatively, on the developer mailing list(<a href="mailto:dev@carbondata.apache.org">dev@carbondata.apache.org</a>).</p>
+<p>If there?s an existing JIRA issue for your intended contribution, please comment about your
+intended work. Once the work is understood, a committer will assign the issue to you.
+(If you don?t have a JIRA role yet, you?ll be added to the ?contributor? role.) If an issue is
+currently assigned, please check with the current assignee before reassigning.</p>
+<p>For moderate or large contributions, you should not start coding or writing a design doc unless
+there is a corresponding JIRA issue assigned to you for that work. Simple changes,
+like fixing typos, do not require an associated issue.</p>
+<h3>
+<a id="design" class="anchor" href="#design" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Design</h3>
+<p>To clearly express your thoughts and get early feedback from other community members, we encourage you to clearly scope, document the design of non-trivial contributions and discuss with the CarbonData community before you start coding.</p>
+<p>Generally, the JIRA issue is the best place to gather relevant design docs, comments, or references. It?s great to explicitly include relevant stakeholders early in the conversation. For designs that may be generally interesting, we also encourage conversations on the developer?s mailing list.</p>
+<h3>
+<a id="code" class="anchor" href="#code" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Code</h3>
+<p>We use GitHub?s pull request functionality to review proposed code changes.
+If you do not already have a personal GitHub account, sign up <a href="https://github.com" target=_blank>here</a>.</p>
+<h3>
+<a id="git-config" class="anchor" href="#git-config" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Git config</h3>
+<p>Ensure to finish the below config(user.email, user.name) before starting PR works.</p>
+<pre><code>$ git config --global user.email "you@example.com"
+$ git config --global user.name "Your Name"
+</code></pre>
+<h4>
+<a id="fork-the-repository-on-github" class="anchor" href="#fork-the-repository-on-github" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Fork the repository on GitHub</h4>
+<p>Go to the <a href="https://github.com/apache/carbondata" target=_blank>Apache CarbonData GitHub mirror</a> and
+fork the repository to your account.
+This will be your private workspace for staging changes.</p>
+<h4>
+<a id="clone-the-repository-locally" class="anchor" href="#clone-the-repository-locally" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Clone the repository locally</h4>
+<p>You are now ready to create the development environment on your local machine.
+Clone CarbonData?s read-only GitHub mirror.</p>
+<pre><code>$ git clone https://github.com/apache/carbondata.git
+$ cd carbondata
+</code></pre>
+<p>Add your forked repository as an additional Git remote, where you?ll push your changes.</p>
+<pre><code>$ git remote add &lt;GitHub_user&gt; https://github.com/&lt;GitHub_user&gt;/carbondata.git
+</code></pre>
+<p>You are now ready to start developing!</p>
+<h4>
+<a id="create-a-branch-in-your-fork" class="anchor" href="#create-a-branch-in-your-fork" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create a branch in your fork</h4>
+<p>You?ll work on your contribution in a branch in your own (forked) repository. Create a local branch,
+initialized with the state of the branch you expect your changes to be merged into.
+Keep in mind that we use several branches, including master, feature-specific, and
+release-specific branches. If you are unsure, initialize with the state of the master branch.</p>
+<pre><code>$ git fetch --all
+$ git checkout -b &lt;my-branch&gt; origin/master
+</code></pre>
+<p>At this point, you can start making and committing changes to this branch in a standard way.</p>
+<h4>
+<a id="syncing-and-pushing-your-branch" class="anchor" href="#syncing-and-pushing-your-branch" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Syncing and pushing your branch</h4>
+<p>Periodically while you work, and certainly before submitting a pull request, you should update
+your branch with the most recent changes to the target branch.</p>
+<pre><code>$ git pull --rebase
+</code></pre>
+<p>Remember to always use --rebase parameter to avoid extraneous merge commits.</p>
+<p>To push your local, committed changes to your (forked) repository on GitHub, run:</p>
+<pre><code>$ git push &lt;GitHub_user&gt; &lt;my-branch&gt;
+</code></pre>
+<h4>
+<a id="testing" class="anchor" href="#testing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Testing</h4>
+<p>All code should have appropriate unit testing coverage. New code should have new tests in the
+same contribution. Bug fixes should include a regression test to prevent the issue from reoccurring.</p>
+<p>For contributions to the Java code, run unit tests locally via Maven.</p>
+<pre><code>$ mvn clean verify
+</code></pre>
+<h3>
+<a id="review" class="anchor" href="#review" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Review</h3>
+<p>Once the initial code is complete and the tests pass, it?s time to start the code review process.
+We review and discuss all code, no matter who authors it. It?s a great way to build community,
+since you can learn from other developers, and they become familiar with your contribution.
+It also builds a strong project by encouraging a high quality bar and keeping code consistent
+throughout the project.</p>
+<h4>
+<a id="create-a-pull-request" class="anchor" href="#create-a-pull-request" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Create a pull request</h4>
+<p>Organize your commits to make your reviewer?s job easier. Use the following command to
+re-order, squash, edit, or change description of individual commits.</p>
+<pre><code>$ git rebase -i origin/master
+</code></pre>
+<p>Navigate to the CarbonData GitHub mirror to create a pull request. The title of the pull request
+should be strictly in the following format:</p>
+<pre><code>[CARBONDATA-JiraTicketNumer][FeatureName] Description of pull request    
+</code></pre>
+<p>Please include a descriptive pull request message to help make the reviewer?s job easier:</p>
+<pre><code> - The root cause/problem statement
+ - What is the implemented solution
+</code></pre>
+<p>If you know a good committer to review your pull request, please make a comment like the following.
+If not, don?t worry, a committer will pick it up.</p>
+<pre><code>Hi @&lt;committer/reviewer name&gt;, can you please take a look?
+</code></pre>
+<h4>
+<a id="code-review-and-revision" class="anchor" href="#code-review-and-revision" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Code Review and Revision</h4>
+<p>During the code review process, don?t rebase your branch or otherwise modify published commits,
+since this can remove existing comment history and be confusing to the reviewer,
+When you make a revision, always push it in a new commit.</p>
+<p>Our GitHub mirror automatically provides pre-commit testing coverage using Jenkins.
+Please make sure those tests pass,the contribution cannot be merged otherwise.</p>
+<h4>
+<a id="lgtm" class="anchor" href="#lgtm" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LGTM</h4>
+<p>Once the reviewer is happy with the change, they?ll respond with an LGTM (?looks good to me!?).
+At this point, the committer will take over, possibly make some additional touch ups,
+and merge your changes into the codebase.</p>
+<p>In the case both the author and the reviewer are committers, either can merge the pull request.
+Just be sure to communicate clearly whose responsibility it is in this particular case.</p>
+<p>Thank you for your contribution to Apache CarbonData!</p>
+<h4>
+<a id="deleting-your-branchoptional" class="anchor" href="#deleting-your-branchoptional" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deleting your branch(optional)</h4>
+<p>Once the pull request is merged into the Apache CarbonData repository, you can safely delete the
+branch locally and purge it from your forked repository.</p>
+<p>From another local branch, run:</p>
+<pre><code>$ git fetch --all
+$ git branch -d &lt;my-branch&gt;
+$ git push &lt;GitHub_user&gt; --delete &lt;my-branch&gt;
+</code></pre>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__contri').addClass('selected'); });
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/index.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/index.html b/src/main/webapp/index.html
index e4e0c0e..e9e6e3b 100644
--- a/src/main/webapp/index.html
+++ b/src/main/webapp/index.html
@@ -70,7 +70,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="">Documentation</a></li>
+                    <li><a href="documentation.html" class="">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button"
                            aria-haspopup="true" aria-expanded="false">Community <span

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/installation-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/installation-guide.html b/src/main/webapp/installation-guide.html
index 2e7fab6..696db00 100644
--- a/src/main/webapp/installation-guide.html
+++ b/src/main/webapp/installation-guide.html
@@ -452,4 +452,4 @@ hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/introduction.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/introduction.html b/src/main/webapp/introduction.html
new file mode 100644
index 0000000..29d2aa1
--- /dev/null
+++ b/src/main/webapp/introduction.html
@@ -0,0 +1,361 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div><h2>
+<a id="what-is-carbondata" class="anchor" href="#what-is-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>What is CarbonData</h2>
+<p>CarbonData is a fully indexed columnar and Hadoop native data-store for processing heavy analytical workloads and detailed queries on big data with Spark SQL. CarbonData allows faster interactive queries over PetaBytes of data.</p>
+<h2>
+<a id="what-does-this-mean" class="anchor" href="#what-does-this-mean" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>What does this mean</h2>
+<p>CarbonData has specially engineered optimizations like multi level indexing, compression and encoding techniques targeted to improve performance of analytical queries which can include filters, aggregation and distinct counts where users expect sub second response time for queries on TB level data on commodity hardware clusters with just a few nodes.</p>
+<p>CarbonData has</p>
+<ul>
+<li>
+<p><strong>Unique data organisation</strong> for faster retrievals and minimise amount of data retrieved</p>
+</li>
+<li>
+<p><strong>Advanced push down optimisations</strong> for deep integration with Spark so as to improvise the Spark DataSource API and other experimental features thereby ensure computing is performed close to the data to minimise amount of data read, processed, converted and transmitted(shuffled)</p>
+</li>
+<li>
+<p><strong>Multi level indexing</strong> to efficiently prune the files and data to be scanned and hence reduce I/O scans and CPU processing</p>
+</li>
+</ul>
+<h2>
+<a id="architecture" class="anchor" href="#architecture" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Architecture</h2>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_architecture.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_architecture.png" alt="" style="max-width:100%;"></a></p>
+<h4>
+<a id="spark-interface-layer" class="anchor" href="#spark-interface-layer" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Spark Interface Layer:</h4>
+<p>CarbonData has deep integration with Apache Spark.CarbonData integrates custom Parser,Strategies,Optimization rules into Spark to take advantage of computing performed closer to data.</p>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_spark_integration.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_spark_integration.png" alt="" style="max-width:100%;"></a></p>
+<ol>
+<li>
+<strong>Carbon parser</strong> Enhances Spark?s SQL parser to support Carbon specific DDL and DML commands to create carbon table, create aggregate tables, manage data loading, data retention and cleanup.</li>
+<li>
+<strong>Carbon Strategies</strong>:- Modify Spark SQL?s physical query execution plan to push down possible operations to Carbon for example:- Grouping, Distinct Count, Top N etc.. for improving query performance.</li>
+<li>
+<strong>Carbon Data RDD</strong>:- Makes the data present in Carbon tables visible to Spark as a RDD which enables spark to perform distributed computation on Carbon tables.</li>
+</ol>
+<h4>
+<a id="carbon-processor" class="anchor" href="#carbon-processor" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Carbon Processor:</h4>
+<p>Receives a query execution fragment from spark and executes the same on the Carbon storage. This involves Scanning the carbon store files for matching record, using the indices to directly locate the row sets and even the rows that may containing the data being searched for. The Carbon processor also performs all pushed down operations such as</p>
+<p>Aggregation/Group By</p>
+<p>Distinct Count</p>
+<p>Top N</p>
+<p>Expression Evaluation</p>
+<p>And many more?</p>
+<h4>
+<a id="carbon-storage" class="anchor" href="#carbon-storage" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Carbon Storage:</h4>
+<p>Custom columnar data store which is heavily compressed, binary, dictionary encoded and heavily indexed.Usaually stored in HDFS.</p>
+<h2>
+<a id="carbondata-features" class="anchor" href="#carbondata-features" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Features</h2>
+<p>CarbonData has rich set of featues to support various use cases in Big Data analytics.</p>
+<h2>
+<a id="design" class="anchor" href="#design" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Design</h2>
+<ul>
+<li>
+<h3>
+<a id="dictionary-encoding" class="anchor" href="#dictionary-encoding" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Dictionary Encoding</h3>
+</li>
+</ul>
+<p>CarbonData supports encoding of data with suggogate values to reduce storage space and speed up processing.Most databases and big data SQL data stores adopt dictionary encoding(integer surrogate numbers) to achieve data compression.Unlike other column store databases where the dictionary is local to each data block, CarbonData maintains a global dictionary which provides opportunity for lazy conversion to actual values enabling all computation to be performed on the lightweight surrogate values.</p>
+<h5>
+<a id="dictionary-generation" class="anchor" href="#dictionary-generation" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Dictionary generation</h5>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_dict_encoding.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_dict_encoding.png" alt="" style="max-width:100%;"></a></p>
+<h5>
+<a id=.htmlk-indexing" class="anchor" href=".htmlk-indexing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>MDK Indexing</h5>
+<p>All the surrogate keys are byte packed to generate an MDK (Multi Dimensional Key) Index.</p>
+<p>Any non surrogate columns of String data types are compressed using one of the configured compression algorithms and stored.For those numeric columns where surrogates are not generated, such data is stored as it is after compression.</p>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk.png" alt="image-20180903212418381" style="max-width:100%;"></a></p>
+<h5>
+<a id="sorted.htmlk" class="anchor" href="#sorted.htmlk" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Sorted MDK</h5>
+<p>The data is sorted based on the MDK Index.Sorting helps for logical grouping of similar data and there by aids in faster look up during query.</p>
+<h4>
+<a id="" class="anchor" href="#" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk_sort.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk_sort.png" alt="image-20180903212525214" style="max-width:100%;"></a>
+</h4>
+<h5>
+<a id="custom-columnar-encoding" class="anchor" href="#custom-columnar-encoding" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Custom Columnar Encoding</h5>
+<p>The Sorted MDK Index is split into each column.Unlike other stores where the column is compressed and stored as it is, CarbonData sorts this column data so that Binary Search can be performed on individual column data based on the filter conditions.This aids in magnitude increase in query performance and also in better compression.Since the individual column's data gets sorted, it is necessary to maintain the row mapping with the sorted MDK Index data in order to retrieve data from other columns which are not participating in filter.This row mapping is termed as <strong>Inverted Index</strong> and is stored along with the column data.The below picture depicts the logical column view.User has the option to <strong>turn off</strong> Inverted Index for such columns where filters are never applied or is very rare.In such cases, scanning would be sequential, but can aid in reducing the storage size(occupied due to inverted index data).</p>
+<h4>
+<a id="-1" class="anchor" href="#-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_blocklet_view.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_blocklet_view.png" alt="" style="max-width:100%;"></a>
+</h4>
+<ul>
+<li>
+<h3>
+<a id="carbondata-storage-format" class="anchor" href="#carbondata-storage-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Storage Format</h3>
+<p>CarbonData has a unique storage structure which aids in efficient storage and retrieval of data.Please refer to <a href="#./file-structure-of-carbondata.html">File Structure of CarbonData</a> for detailed information on the format.</p>
+</li>
+<li>
+<h3>
+<a id="indexing" class="anchor" href="#indexing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Indexing</h3>
+<p>CarbonData maintains multiple indexes at multiple levels to assist in efficient pruning of unwanted data from scan during query.Also CarbonData has support for plugging in external indexing solutions to speed up the query process.</p>
+<h5>
+<a id="min-max-indexing" class="anchor" href="#min-max-indexing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Min-Max Indexing</h5>
+<p>Storing data along with index significantly accelerates query performance and reduces the I/O scans and CPU resources in case of filters in the query. CarbonData index consists of multiple levels of indices, a processing framework can leverage this index to reduce the number of tasks it needs to schedule and process. It can also do skip scan in more fine grained units (called blocklet) in task side scanning instead of scanning the whole file.  <strong>CarbonData maintains Min-Max Index for all the columns.</strong></p>
+<p>CarbonData maintains a separate index file which contains the footer information for efficient IO reads.</p>
+<p>Using the Min-Max info in these index files, two levels of filtering can be achieved.</p>
+<p>Min-Max at the carbondata file level,to efficiently prune the files when the filter condition doesn't fall in the range.This information when maintained at the Spark Driver, will help to efficiently schedule the tasks for scanning</p>
+<p>Min-Max at the blocklet level, to efficiently prune the blocklets when the filter condition doesn't fall in the range.This information when maintained at the executor can significantly reduce the amount unnecessary data processed by the executor tasks.</p>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-minmax-blocklet.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-minmax-blocklet.png" alt="" style="max-width:100%;"></a></p>
+</li>
+<li>
+<h4>
+<a id="datamaps" class="anchor" href="#datamaps" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DataMaps</h4>
+<p>DataMap is a framework for indexing and also for statistics that can be used to add primary index (Blocklet Index) , secondary index type and statistical type to CarbonData.</p>
+<p>DataMap is a standardized general interface which CarbonData uses to prune data blocks for scanning.</p>
+<p>DataMaps are of 2 types:</p>
+<p><strong>CG(Coarse Grained) DataMaps</strong> Can prune data to the blocklet or to Page level.ie., Holds information for deciding which blocks/blocklets to be scanned.This DataMap is used in Spark Driver to decide the number of tasks to be scheduled.</p>
+<p><strong>FG(Fine Grained) DataMaps</strong> Can prune data to row level.This DataMap is used in Spark executor for scanning an fetching the data much faster.</p>
+<p>Since DataMap interfaces are generalised, We can write a thin adaptor called as <strong>DataMap Providers</strong> to interface between CarbonData and other external Indexing engines. For eg., Lucene, Solr,ES,...</p>
+<p>CarbonData has its own DSL to create and manage DataMaps.Please refer to <a href="#./datamap/datamap-management.html#overview">CarbonData DSL</a> for more information.</p>
+<p>The below diagram explains about the DataMap execution in CarbonData.</p>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-datamap.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-datamap.png" alt="" style="max-width:100%;"></a></p>
+</li>
+<li>
+<h4>
+<a id="update--delete" class="anchor" href="#update--delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Update &amp; Delete</h4>
+</li>
+</ul>
+<p>CarbonData supports Update and delete operations over big data.This functionality is not targetted for OLTP scenarios where high concurrent update/delete are required.Following are the assumptions considered when this feature is designed.</p>
+<ol>
+<li>Updates or Deletes are periodic and in Bulk</li>
+<li>Updates or Deletes are atomic</li>
+<li>Data is immediately visible</li>
+<li>Concurrent query to be allowed during an update or delete operation</li>
+<li>Single statement auto-commit support (not OLTP-style transaction)</li>
+</ol>
+<p>Since data stored in HDFS are immutable,data blocks cannot be updated in-place.Re-write of entire data block is not efficient for IO and also is a slow process.</p>
+<p>To over come these limitations, CarbonData adopts methodology of writing a delta file containing the rows to be deleted and another delta file containing the values to be updated with.During processing, These two delta files are merged with the main carbondata file and the correct result is returned for the query.</p>
+<p>The below diagram describes the process.</p>
+<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_update_delete.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_update_delete.png" alt="" style="max-width:100%;"></a></p>
+<h2>
+<a id="integration-with-big-data-ecosystem" class="anchor" href="#integration-with-big-data-ecosystem" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Integration with Big Data ecosystem</h2>
+<p>Refer to Integration with <a href="#./quick-start-guide.html#spark">Spark</a>, <a href="#./quick-start-guide.html#presto">Presto</a> for detailed information on integrating CarbonData with these execution engines.</p>
+<h2>
+<a id="scenarios-where-carbondata-is-suitable" class="anchor" href="#scenarios-where-carbondata-is-suitable" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Scenarios where CarbonData is suitable</h2>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__intro').addClass('selected'); });
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/language-manual.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/language-manual.html b/src/main/webapp/language-manual.html
new file mode 100644
index 0000000..54c29ac
--- /dev/null
+++ b/src/main/webapp/language-manual.html
@@ -0,0 +1,273 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="overview" class="anchor" href="#overview" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Overview</h1>
+<p>CarbonData has its own parser, in addition to Spark's SQL Parser, to parse and process certain Commands related to CarbonData table handling. You can interact with the SQL interface using the <a href="https://spark.apache.org/docs/latest/sql-programming-guide.html#running-the-spark-sql-cli" rel="nofollow">command-line</a> or over <a href="https://spark.apache.org/docs/latest/sql-programming-guide.html#running-the-thrift-jdbcodbc-server" rel="nofollow">JDBC/ODBC</a>.</p>
+<ul>
+<li><a href="./supported-data-types-in-carbondata.html">Data Types</a></li>
+<li>Data Definition Statements
+<ul>
+<li>
+<a href="./ddl-of-carbondata.html">DDL:</a><a href="./ddl-of-carbondata.html#create-table">Create</a>,<a href="./ddl-of-carbondata.html#drop-table">Drop</a>,<a href="./ddl-of-carbondata.html#partition">Partition</a>,<a href="./ddl-of-carbondata.html#bucketing">Bucketing</a>,<a href="./ddl-of-carbondata.html#alter-table">Alter</a>,<a href="./ddl-of-carbondata.html#create-table-as-select">CTAS</a>,<a href="./ddl-of-carbondata.html#create-external-table">External Table</a>
+</li>
+<li>Indexes</li>
+<li>
+<a href="./datamap-management.html">DataMaps</a>
+<ul>
+<li><a href="./bloomfilter-datamap-guide.html">Bloom</a></li>
+<li><a href="./lucene-datamap-guide.html">Lucene</a></li>
+<li><a href="./preaggregate-datamap-guide.html">Pre-Aggregate</a></li>
+<li><a href="./timeseries-datamap-guide.html">Time Series</a></li>
+</ul>
+</li>
+<li>Materialized Views (MV)</li>
+<li><a href="./streaming-guide.html">Streaming</a></li>
+</ul>
+</li>
+<li>Data Manipulation Statements
+<ul>
+<li>
+<a href="./dml-of-carbondata.html">DML:</a> <a href="./dml-of-carbondata.html#load-data">Load</a>, <a href="./ddl-of-carbondata.html#insert-overwrite">Insert</a>, <a href="./dml-of-carbondata.html#update">Update</a>, <a href="./dml-of-carbondata.html#delete">Delete</a>
+</li>
+<li><a href="./segment-management-on-carbondata.html">Segment Management</a></li>
+</ul>
+</li>
+<li><a href="./configuration-parameters.html">Configuration Properties</a></li>
+</ul>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/main/webapp/lucene-datamap-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/lucene-datamap-guide.html b/src/main/webapp/lucene-datamap-guide.html
index 7b1598e..038cb9d 100644
--- a/src/main/webapp/lucene-datamap-guide.html
+++ b/src/main/webapp/lucene-datamap-guide.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -302,6 +353,17 @@ release, user can do as following:</p>
 <li>Create the lucene datamap again by <code>CREATE DATAMAP</code> command.
 Basically, user can manually trigger the operation by re-building the datamap.</li>
 </ol>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -317,4 +379,4 @@ Basically, user can manually trigger the operation by re-building the datamap.</
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>


[05/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/src/site/markdown/configuration-parameters.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/configuration-parameters.md b/src/site/markdown/configuration-parameters.md
index 46b8bd0..de72439 100644
--- a/src/site/markdown/configuration-parameters.md
+++ b/src/site/markdown/configuration-parameters.md
@@ -7,7 +7,7 @@
     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.
@@ -16,152 +16,135 @@
 -->
 
 # Configuring CarbonData
- This tutorial guides you through the advanced configurations of CarbonData :
- 
+ This guide explains the configurations that can be used to tune CarbonData to achieve better performance.Some of the properties can be set dynamically and are explained in the section Dynamic Configuration In CarbonData Using SET-RESET.Most of the properties that control the internal settings have reasonable default values.They are listed along with the properties along with explanation.
+
  * [System Configuration](#system-configuration)
- * [Performance Configuration](#performance-configuration)
- * [Miscellaneous Configuration](#miscellaneous-configuration)
- * [Spark Configuration](#spark-configuration)
+ * [Data Loading Configuration](#data-loading-configuration)
+ * [Compaction Configuration](#compaction-configuration)
+ * [Query Configuration](#query-configuration)
+ * [Data Mutation Configuration](#data-mutation-configuration)
  * [Dynamic Configuration In CarbonData Using SET-RESET](#dynamic-configuration-in-carbondata-using-set-reset)
- 
- 
+
+
 ##  System Configuration
 This section provides the details of all the configurations required for the CarbonData System.
 
-<b><p align="center">System Configuration in carbon.properties</p></b>
-
 | Property | Default Value | Description |
 |----------------------------|-------------------------------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| carbon.storelocation |  | Location where CarbonData will create the store, and write the data in its own format. If not specified then it takes spark.sql.warehouse.dir path. NOTE: Store location should be in HDFS. |
-| carbon.ddl.base.hdfs.url |  | This property is used to configure the HDFS relative path, the path configured in carbon.ddl.base.hdfs.url will be appended to the HDFS path configured in fs.defaultFS. If this path is configured, then user need not pass the complete path while dataload. For example: If absolute path of the csv file is hdfs://10.18.101.155:54310/data/cnbc/2016/xyz.csv, the path "hdfs://10.18.101.155:54310" will come from property fs.defaultFS and user can configure the /data/cnbc/ as carbon.ddl.base.hdfs.url. Now while dataload user can specify the csv path as /2016/xyz.csv. |
-| carbon.badRecords.location |  | Path where the bad records are stored. |
-| carbon.data.file.version | V3 | If this parameter value is set to 1, then CarbonData will support the data load which is in old format(0.x version). If the value is set to 2(1.x onwards version), then CarbonData will support the data load of new format only.|
-| carbon.streaming.auto.handoff.enabled | true | If this parameter value is set to true, auto trigger handoff function will be enabled.|
-| carbon.streaming.segment.max.size | 1024000000 | This parameter defines the maximum size of the streaming segment. Setting this parameter to appropriate value will avoid impacting the streaming ingestion. The value is in bytes.|
-| carbon.query.show.datamaps | true | If this parameter value is set to true, show tables command will list all the tables including datatmaps(eg: Preaggregate table), else datamaps will be excluded from the table list. |
-| carbon.segment.lock.files.preserve.hours | 48 | This property value indicates the number of hours the segment lock files will be preserved after dataload. These lock files will be deleted with the clean command after the configured number of hours. |
-| carbon.unsafe.working.memory.in.mb | 512 | Specifies the size of executor unsafe working memory. Used for sorting data, storing column pages,etc. This value is expressed in MB. |
-| carbon.unsafe.driver.working.memory.in.mb | 512 | Specifies the size of driver unsafe working memory. Used for storing block or blocklet datamap cache. If not configured then carbon.unsafe.working.memory.in.mb value is considered. This value is expressed in MB. |
-
-##  Performance Configuration
-This section provides the details of all the configurations required for CarbonData Performance Optimization.
-
-<b><p align="center">Performance Configuration in carbon.properties</p></b>
-
-* **Data Loading Configuration**
-
-| Parameter | Default Value | Description | Range |
-|--------------------------------------|---------------|----------------------------------------------------------------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| carbon.number.of.cores.while.loading | 2 | Number of cores to be used while loading data. |  |
-| carbon.sort.size | 100000 | Record count to sort and write intermediate files to temp. |  |
-| carbon.max.driver.lru.cache.size | -1 | Max LRU cache size upto which data will be loaded at the driver side. This value is expressed in MB. Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted. |  |
-| carbon.max.executor.lru.cache.size | -1 | Max LRU cache size upto which data will be loaded at the executor side. This value is expressed in MB. Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted. If this parameter is not configured, then the carbon.max.driver.lru.cache.size value will be considered. |  |
-| carbon.merge.sort.prefetch | true | Enable prefetch of data during merge sort while reading data from sort temp files in data loading. |  |
-| carbon.insert.persist.enable | false | Enabling this parameter considers persistent data. If we are executing insert into query from source table using select statement & loading the same source table concurrently, when select happens on source table during the data load, it gets new record for which dictionary is not generated, so there will be inconsistency. To avoid this condition we can persist the dataframe into MEMORY_AND_DISK(default value) and perform insert into operation. By default this value will be false because no need to persist the dataframe in all cases. If user wants to run load and insert queries on source table concurrently then user can enable this parameter. |  |
-| carbon.insert.storage.level | MEMORY_AND_DISK | Which storage level to persist dataframe when 'carbon.insert.persist.enable'=true, if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. [See detail](http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence). |  |
-| carbon.update.persist.enable | true | Enabling this parameter considers persistent data. Enabling this will reduce the execution time of UPDATE operation. |  |
-| carbon.update.storage.level | MEMORY_AND_DISK | Which storage level to persist dataframe when 'carbon.update.persist.enable'=true, if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. [See detail](http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence). |  |
-| carbon.global.sort.rdd.storage.level | MEMORY_ONLY | Which storage level to persist rdd when loading data with 'sort_scope'='global_sort', if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. [See detail](http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence). |  |
-| carbon.load.global.sort.partitions | 0 | The Number of partitions to use when shuffling data for sort. If user don't configurate or configurate it less than 1, it uses the number of map tasks as reduce tasks. In general, we recommend 2-3 tasks per CPU core in your cluster.
-| carbon.options.bad.records.logger.enable | false | Whether to create logs with details about bad records. | |
-| carbon.bad.records.action | FORCE | This property can have four types of actions for bad records FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found. | |
-| carbon.options.is.empty.data.bad.record | false | If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa. | |
-| carbon.options.bad.record.path |  | Specifies the HDFS path where bad records are stored. By default the value is Null. This path must to be configured by the user if bad record logger is enabled or bad record action redirect. | |
-| carbon.enable.vector.reader | true | This parameter increases the performance of select queries as it fetch columnar batch of size 4*1024 rows instead of fetching data row by row. | |
-| carbon.blockletgroup.size.in.mb | 64 MB | The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of the blocklet group. Higher value results in better sequential IO access.The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB). |  |
-| carbon.task.distribution | block | **block**: Setting this value will launch one task per block. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. **custom**: Setting this value will group the blocks and distribute it uniformly to the available resources in the cluster. This enhances the query performance but not suggested in case of concurrent queries and queries having big shuffling scenarios. **blocklet**: Setting this value will launch one task per blocklet. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. **merge_small_files**: Setting this value will merge all the small partitions to a size of (128 MB is the default value of "spark.sql.files.maxPartitionBytes",it is configurable) during querying. The small partitions are combined to a map task to reduce the number of read task. This enhances the performance. | |
-| carbon.load.sortmemory.spill.percentage | 0 | If we use unsafe memory during data loading, this configuration will be used to control the behavior of spilling inmemory pages to disk. Internally in Carbondata, during sorting carbondata will sort data in pages and add them in unsafe memory. If the memory is insufficient, carbondata will spill the pages to disk and generate sort temp file. This configuration controls how many pages in memory will be spilled to disk based size. The size can be calculated by multiplying this configuration value with 'carbon.sort.storage.inmemory.size.inmb'. For example, default value 0 means that no pages in unsafe memory will be spilled and all the newly sorted data will be spilled to disk; Value 50 means that if the unsafe memory is insufficient, about half of pages in the unsafe memory will be spilled to disk while value 100 means that almost all pages in unsafe memory will be spilled. **Note**: This configuration only works for 'LOCAL_SORT' and 'BA
 TCH_SORT' and the actual spilling behavior may slightly be different in each data loading. | Integer values between 0 and 100 |
-
-* **Compaction Configuration**
-  
-| Parameter | Default Value | Description | Range |
-|-----------------------------------------------|---------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------|
-| carbon.number.of.cores.while.compacting | 2 | Number of cores which are used to write data during compaction. |  |
-| carbon.compaction.level.threshold | 4, 3 | This property is for minor compaction which decides how many segments to be merged. Example: If it is set as 2, 3 then minor compaction will be triggered for every 2 segments. 3 is the number of level 1 compacted segment which is further compacted to new segment. | Valid values are from 0-100. |
-| carbon.major.compaction.size | 1024 | Major compaction size can be configured using this parameter. Sum of the segments which is below this threshold will be merged. This value is expressed in MB. |  |
-| carbon.horizontal.compaction.enable | true | This property is used to turn ON/OFF horizontal compaction. After every DELETE and UPDATE statement, horizontal compaction may occur in case the delta (DELETE/ UPDATE) files becomes more than specified threshold. |  |
-| carbon.horizontal.UPDATE.compaction.threshold | 1 | This property specifies the threshold limit on number of UPDATE delta files within a segment. In case the number of delta files goes beyond the threshold, the UPDATE delta files within the segment becomes eligible for horizontal compaction and compacted into single UPDATE delta file. | Values between 1 to 10000. |
-| carbon.horizontal.DELETE.compaction.threshold | 1 | This property specifies the threshold limit on number of DELETE delta files within a block of a segment. In case the number of delta files goes beyond the threshold, the DELETE delta files for the particular block of the segment becomes eligible for horizontal compaction and compacted into single DELETE delta file. | Values between 1 to 10000. |
-| carbon.update.segment.parallelism | 1 | This property specifies the parallelism for each segment during update. If there are segments that contain too many records to update and the spark job encounter data-spill related errors, it is better to increase this property value. It is recommended to set this value to a multiple of the number of executors for balance. | Values between 1 to 1000. |
-| carbon.merge.index.in.segment | true | This property is used to merge all carbon index files (.carbonindex) inside a segment to a single carbon index merge file (.carbonindexmerge).| Values true or false |  
-
-* **Query Configuration**
-  
-| Parameter | Default Value | Description | Range |
-|--------------------------------------|---------------|---------------------------------------------------|---------------------------|
-| carbon.number.of.cores | 4 | Number of cores to be used while querying. |  |
-| carbon.enable.quick.filter | false | Improves the performance of filter query. |  |
-
-
-##   Miscellaneous Configuration
-
-<b><p align="center">Extra Configuration in carbon.properties</p></b>
-
-* **Time format for CarbonData** 
-
-| Parameter | Default Format | Description |
-|-------------------------|---------------------|--------------------------------------------------------------|
-| carbon.timestamp.format | yyyy-MM-dd HH:mm:ss | Timestamp format of input data used for timestamp data type. |
-
-* **Dataload Configuration**
-  
-| Parameter | Default Value | Description |
-|---------------------------------------------|--------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| carbon.sort.file.write.buffer.size | 16384 | File write buffer size used during sorting. Minimum allowed buffer size is 10240 byte and Maximum allowed buffer size is 10485760 byte. |
+| carbon.storelocation | spark.sql.warehouse.dir property value | Location where CarbonData will create the store, and write the data in its custom format. If not specified,the path defaults to spark.sql.warehouse.dir property. NOTE: Store location should be in HDFS. |
+| carbon.ddl.base.hdfs.url | (none) | To simplify and shorten the path to be specified in DDL/DML commands, this property is supported.This property is used to configure the HDFS relative path, the path configured in carbon.ddl.base.hdfs.url will be appended to the HDFS path configured in fs.defaultFS of core-site.xml. If this path is configured, then user need not pass the complete path while dataload. For example: If absolute path of the csv file is hdfs://10.18.101.155:54310/data/cnbc/2016/xyz.csv, the path "hdfs://10.18.101.155:54310" will come from property fs.defaultFS and user can configure the /data/cnbc/ as carbon.ddl.base.hdfs.url. Now while dataload user can specify the csv path as /2016/xyz.csv. |
+| carbon.badRecords.location | (none) | CarbonData can detect the records not conforming to defined table schema and isolate them as bad records.This property is used to specify where to store such bad records. |
+| carbon.streaming.auto.handoff.enabled | true | CarbonData supports storing of streaming data.To have high throughput for streaming, the data is written in Row format which is highly optimized for write, but performs poorly for query.When this property is true and when the streaming data size reaches ***carbon.streaming.segment.max.size***, CabonData will automatically convert the data to columnar format and optimize it for faster querying.**NOTE:** It is not recommended to keep the default value which is true. |
+| carbon.streaming.segment.max.size | 1024000000 | CarbonData writes streaming data in row format which is optimized for high write throughput.This property defines the maximum size of data to be held is row format, beyond which it will be converted to columnar format in order to support high performane query, provided ***carbon.streaming.auto.handoff.enabled*** is true. **NOTE:** Setting higher value will impact the streaming ingestion. The value has to be configured in bytes. |
+| carbon.query.show.datamaps | true | CarbonData stores datamaps as independent tables so as to allow independent maintenance to some extent.When this property is true,which is by default, show tables command will list all the tables including datatmaps(eg: Preaggregate table), else datamaps will be excluded from the table list.**NOTE:**  It is generally not required for the user to do any maintenance operations on these tables and hence not required to be seen.But it is shown by default so that user or admin can get clear understanding of the system for capacity planning. |
+| carbon.segment.lock.files.preserve.hours | 48 | In order to support parallel data loading onto the same table, CarbonData sequences(locks) at the granularity of segments.Operations affecting the segment(like IUD, alter) are blocked from parallel operations.This property value indicates the number of hours the segment lock files will be preserved after dataload. These lock files will be deleted with the clean command after the configured number of hours. |
+| carbon.timestamp.format | yyyy-MM-dd HH:mm:ss | CarbonData can understand data of timestamp type and process it in special manner.It can be so that the format of Timestamp data is different from that understood by CarbonData by default.This configuration allows users to specify the format of Timestamp in their data. |
 | carbon.lock.type | LOCALLOCK | This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: 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: 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 HDFS supports file based locking. |
-| carbon.lock.path | TABLEPATH | Locks on the files are used to prevent concurrent operation from modifying the same files. This 
-configuration specifies the path where lock files have to be created. Recommended to configure 
-HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3. 
-**Note:** If this property is not set to HDFS location for S3 store, then there is a possibility 
-of data corruption because multiple data manipulation calls might try to update the status file 
-and as lock is not acquired before updation data might get overwritten. |
-| carbon.sort.intermediate.files.limit | 20 | Minimum number of intermediate files after which merged sort can be started (minValue = 2, maxValue=50). |
-| carbon.block.meta.size.reserved.percentage | 10 | Space reserved in percentage for writing block meta data in CarbonData file. |
-| carbon.csv.read.buffersize.byte | 1048576 | csv reading buffer size. |
-| carbon.merge.sort.reader.thread | 3 | Maximum no of threads used for reading intermediate files for final merging. |
-| carbon.concurrent.lock.retries | 100 | Specifies the maximum number of retries to obtain the lock for concurrent operations. This is used for concurrent loading. |
-| carbon.concurrent.lock.retry.timeout.sec | 1 | Specifies the interval between the retries to obtain the lock for concurrent operations. |
-| carbon.lock.retries | 3 | Specifies the maximum number of retries to obtain the lock for any operations other than load. |
-| carbon.lock.retry.timeout.sec | 5 | Specifies the interval between the retries to obtain the lock for any operation other than load. |
-| carbon.skip.empty.line | false | Setting this property ignores the empty lines in the CSV file during the data load |
-| carbon.enable.calculate.size | true | **For Load Operation**: Setting this property calculates the size of the carbon data file (.carbondata) and carbon index file (.carbonindex) for every load and updates the table status file. **For Describe Formatted**: Setting this property calculates the total size of the carbon data files and carbon index files for the respective table and displays in describe formatted command. | 
-
-
-
-* **Compaction Configuration**
+| carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3. |
+| carbon.unsafe.working.memory.in.mb | 512 | CarbonData supports storing data in off-heap memory for certain operations during data loading and query.This helps to avoid the Java GC and thereby improve the overall performance.The Minimum value recommeded is 512MB.Any value below this is reset to default value of 512MB.**NOTE:** The below formulas explain how to arrive at the off-heap size required.<u>Memory Required For Data Loading:</u>(*carbon.number.of.cores.while.loading*) * (Number of tables to load in parallel) * (*offheap.sort.chunk.size.inmb* + *carbon.blockletgroup.size.in.mb* + *carbon.blockletgroup.size.in.mb*/3.5 ). <u>Memory required for Query:</u>SPARK_EXECUTOR_INSTANCES * (*carbon.blockletgroup.size.in.mb* + *carbon.blockletgroup.size.in.mb* * 3.5) * spark.executor.cores |
+| carbon.update.sync.folder | /tmp/carbondata | CarbonData maintains last modification time entries in modifiedTime.mdt to determine the schema changes and reload only when necessary.This configuration specifies the path where the file needs to be written. |
+| carbon.invisible.segments.preserve.count | 200 | CarbonData maintains each data load entry in tablestatus file. The entries from this file are not deleted for those segments that are compacted or dropped, but are made invisible.If the number of data loads are very high, the size and number of entries in tablestatus file can become too many causing unnecessary reading of all data.This configuration specifies the number of segment entries to be maintained afte they are compacted or dropped.Beyond this, the entries are moved to a separate history tablestatus file.**NOTE:** The entries in tablestatus file help to identify the operations performed on CarbonData table and is also used for checkpointing during various data manupulation operations.This is similar to AUDIT file maintaining all the operations and its status.Hence the entries are never deleted but moved to a separate history file. |
+| carbon.lock.retries | 3 | CarbonData ensures consistency of operations by blocking certain operations from running in parallel.In order to block the operations from running in parallel, lock is obtained on the table.This configuration specifies the maximum number of retries to obtain the lock for any operations other than load.**NOTE:** Data manupulation operations like Compaction,UPDATE,DELETE  or LOADING,UPDATE,DELETE are not allowed to run in parallel.How ever data loading can happen in parallel to compaction. |
+| carbon.lock.retry.timeout.sec | 5 | Specifies the interval between the retries to obtain the lock for any operation other than load.**NOTE:** Refer to ***carbon.lock.retries*** for understanding why CarbonData uses locks for operations. |
+
+## Data Loading Configuration
 
 | Parameter | Default Value | Description |
-|-----------------------------------|---------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| carbon.numberof.preserve.segments | 0 | If the user wants to preserve some number of segments from being compacted then he can set this property. Example: carbon.numberof.preserve.segments = 2 then 2 latest segments will always be excluded from the compaction. No segments will be preserved by default. |
-| carbon.allowed.compaction.days | 0 | Compaction will merge the segments which are loaded with in the specific number of days configured. Example: If the configuration is 2, then the segments which are loaded in the time frame of 2 days only will get merged. Segments which are loaded 2 days apart will not be merged. This is disabled by default. |
-| carbon.enable.auto.load.merge | false | To enable compaction while data loading. |
-|carbon.enable.page.level.reader.in.compaction|true|Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory.|
+|--------------------------------------|---------------|----------------------------------------------------------------------------------------------------------------------|
+| carbon.number.of.cores.while.loading | 2 | Number of cores to be used while loading data.This also determines the number of threads to be used to read the input files (csv) in parallel.**NOTE:** This configured value is used in every data loading step to parallelize the operations. Configuring a higher value can lead to increased early thread pre-emption by OS and there by reduce the overall performance. |
+| carbon.sort.size | 100000 | Number of records to hold in memory to sort and write intermediate temp files.**NOTE:** Memory required for data loading increases with increase in configured value as each thread would cache configured number of records. |
+| carbon.global.sort.rdd.storage.level | MEMORY_ONLY | Storage level to persist dataset of RDD/dataframe when loading data with 'sort_scope'='global_sort', if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. [See detail](http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence). |
+| carbon.load.global.sort.partitions | 0 | The Number of partitions to use when shuffling data for sort. Default value 0 means to use same number of map tasks as reduce tasks.**NOTE:** In general, it is recommended to have 2-3 tasks per CPU core in your cluster. |
+| carbon.options.bad.records.logger.enable | false | CarbonData can identify the records that are not conformant to schema and isolate them as bad records.Enabling this configuration will make CarbonData to log such bad records.**NOTE:** If the input data contains many bad records, logging them will slow down the over all data loading throughput.The data load operation status would depend on the configuration in ***carbon.bad.records.action***. |
+| carbon.bad.records.action | FAIL | CarbonData in addition to identifying the bad records, can take certain actions on such data.This configuration can have four types of actions for bad records namely FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found. |
+| carbon.options.is.empty.data.bad.record | false | Based on the business scenarios, empty("" or '' or ,,) data can be valid or invalid. This configuration controls how empty data should be treated by CarbonData. If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa. |
+| carbon.options.bad.record.path | (none) | Specifies the HDFS path where bad records are to be stored. By default the value is Null. This path must to be configured by the user if ***carbon.options.bad.records.logger.enable*** is **true** or ***carbon.bad.records.action*** is **REDIRECT**. |
+| carbon.blockletgroup.size.in.mb | 64 | Please refer to [file-structure-of-carbondata](./file-structure-of-carbondata.md ) to understand the storage format of CarbonData.The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of each blocklet group. Higher value results in better sequential IO access.The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB).**NOTE:** Configuring a higher value might lead to poor performance as an entire blocklet group will have to read into memory before processing.For filter queries with limit, it is **not advisable** to have a bigger blocklet size.For Aggregation queries which need to return more number of rows,bigger blocklet size is advisable. |
+| carbon.sort.file.write.buffer.size | 16384 | CarbonData sorts and writes data to intermediate files to limit the memory usage.This configuration determines the buffer size to be used for reading and writing such files. **NOTE:** This configuration is useful to tune IO and derive optimal performance.Based on the OS and underlying harddisk type, these values can significantly affect the overall performance.It is ideal to tune the buffersize equivalent to the IO buffer size of the OS.Recommended range is between 10240 to 10485760 bytes. |
+| carbon.sort.intermediate.files.limit | 20 | CarbonData sorts and writes data to intermediate files to limit the memory usage.Before writing the target carbondat file, the data in these intermediate files needs to be sorted again so as to ensure the entire data in the data load is sorted.This configuration determines the minimum number of intermediate files after which merged sort is applied on them sort the data.**NOTE:** Intermediate merging happens on a separate thread in the background.Number of threads used is determined by ***carbon.merge.sort.reader.thread***.Configuring a low value will cause more time to be spent in merging these intermediate merged files which can cause more IO.Configuring a high value would cause not to use the idle threads to do intermediate sort merges.Range of recommended values are between 2 and 50 |
+| carbon.csv.read.buffersize.byte | 1048576 | CarbonData uses Hadoop InputFormat to read the csv files.This configuration value is used to pass buffer size as input for the Hadoop MR job when reading the csv files.This value is configured in bytes.**NOTE:** Refer to ***org.apache.hadoop.mapreduce.InputFormat*** documentation for additional information. |
+| carbon.merge.sort.reader.thread | 3 | CarbonData sorts and writes data to intermediate files to limit the memory usage.When the intermediate files reaches ***carbon.sort.intermediate.files.limit*** the files will be merged,the number of threads specified in this configuration will be used to read the intermediate files for performing merge sort.**NOTE:** Refer to ***carbon.sort.intermediate.files.limit*** for operation description.Configuring less  number of threads can cause merging to slow down over loading process where as configuring more number of threads can cause thread contention with threads in other data loading steps.Hence configure a fraction of ***carbon.number.of.cores.while.loading***. |
+| carbon.concurrent.lock.retries | 100 | CarbonData supports concurrent data loading onto same table.To ensure the loading status is correctly updated into the system,locks are used to sequence the status updation step.This configuration specifies the maximum number of retries to obtain the lock for updating the load status.**NOTE:** This value is high as more number of concurrent loading happens,more the chances of not able to obtain the lock when tried.Adjust this value according to the number of concurrent loading to be supported by the system. |
+| carbon.concurrent.lock.retry.timeout.sec | 1 | Specifies the interval between the retries to obtain the lock for concurrent operations.**NOTE:** Refer to ***carbon.concurrent.lock.retries*** for understanding why CarbonData uses locks during data loading operations. |
+| carbon.skip.empty.line | false | The csv files givent to CarbonData for loading can contain empty lines.Based on the business scenario, this empty line might have to be ignored or needs to be treated as NULL value for all columns.In order to define this business behavior, this configuration is provided.**NOTE:** In order to consider NULL values for non string columns and continue with data load, ***carbon.bad.records.action*** need to be set to **FORCE**;else data load will be failed as bad records encountered. |
+| carbon.enable.calculate.size | true | **For Load Operation**: Setting this property calculates the size of the carbon data file (.carbondata) and carbon index file (.carbonindex) for every load and updates the table status file. **For Describe Formatted**: Setting this property calculates the total size of the carbon data files and carbon index files for the respective table and displays in describe formatted command.**NOTE:** This is useful to determine the overall size of the carbondata table and also get an idea of how the table is growing in order to take up other backup strategy decisions. |
+| carbon.cutOffTimestamp | (none) | CarbonData has capability to generate the Dictionary values for the timestamp columns from the data itself without the need to store the computed dictionary values. This configuration sets the start date for calculating the timestamp. Java counts the number of milliseconds from start of "1970-01-01 00:00:00". This property is used to customize the start of position. For example "2000-01-01 00:00:00". **NOTE:** The date must be in the form ***carbon.timestamp.format***. CarbonData supports storing data for upto 68 years.For example, if the cut-off time is 1970-01-01 05:30:00, then data upto 2038-01-01 05:30:00 will be supported by CarbonData. |
+| carbon.timegranularity | SECOND | The configuration is used to specify the data granularity level such as DAY, HOUR, MINUTE, or SECOND.This helps to store more than 68 years of data into CarbonData. |
+| carbon.use.local.dir | false | CarbonData during data loading, writes files to local temp directories before copying the files to HDFS.This configuration is used to specify whether CarbonData can write locally to tmp directory of the container or to the YARN application directory. |
+| carbon.use.multiple.temp.dir | false | When multiple disks are present in the system, YARN is generally configured with multiple disks to be used as temp directories for managing the containers.This configuration specifies whether to use multiple YARN local directories during data loading for disk IO load balancing.Enable ***carbon.use.local.dir*** for this configuration to take effect.**NOTE:** Data Loading is an IO intensive operation whose performance can be limited by the disk IO threshold, particularly during multi table concurrent data load.Configuring this parameter, balances the disk IO across multiple disks there by improving the over all load performance. |
+| carbon.sort.temp.compressor | (none) | CarbonData writes every ***carbon.sort.size*** number of records to intermediate temp files during data loading to ensure memory footprint is within limits.These temporary files cab be compressed and written in order to save the storage space.This configuration specifies the name of compressor to be used to compress the intermediate sort temp files during sort procedure in data loading.The valid values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files.**NOTE:** Compressor will be useful if you encounter disk bottleneck.Since the data needs to be compressed and decompressed,it involves additional CPU cycles,but is compensated by the high IO throughput due to less data to be written or read from the disks. |
+| carbon.load.skewedDataOptimization.enabled | false | During data loading,CarbonData would divide the number of blocks equally so as to ensure all executors process same number of blocks.This mechanism satisfies most of the scenarios and ensures maximum parallel processing for optimal data loading performance.In some business scenarios, there might be scenarios where the size of blocks vary significantly and hence some executors would have to do more work if they get blocks containing more data. This configuration enables size based block allocation strategy for data loading.When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data.**NOTE:** This configuration is useful if the size of your input data files varies widely, say 1MB~1GB.For this configuration to work effectively,knowing the data pattern and size is important and necessary. |
+| carbon.load.min.size.enabled | false | During Data Loading, CarbonData would divide the number of files among the available executors to parallelize the loading operation.When the input data files are very small, this action causes to generate many small carbondata files.This configuration determines whether to enable node minumun input data size allocation strategy for data loading.It will make sure that the node load the minimum amount of data there by reducing number of carbondata files.**NOTE:** This configuration is useful if the size of the input data files are very small, like 1MB~256MB.Refer to ***load_min_size_inmb*** to configure the minimum size to be considered for splitting files among executors. |
+| enable.data.loading.statistics | false | CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues.This configuration when made ***true*** would log additional data loading statistics information to more accurately locate the issues being debugged.**NOTE:** Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time.It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately.Also extensive logging is an increased IO operation and hence over all data loading performance might get reduced.Therefore it is recommened to enable this configuration only for the duration of debugging. |
+| carbon.dictionary.chunk.size | 10000 | CarbonData generates dictionary keys and writes them to separate dictionary file during data loading.To optimize the IO, this configuration determines the number of dictionary keys to be persisted to dictionary file at a time.**NOTE:** Writing to file also serves as a commit point to the dictionary generated.Increasing more values in memory causes more data loss during system or application failure.It is advised to alter this configuration judiciously. |
+| dictionary.worker.threads | 1 | CarbonData supports Optimized data loading by relying on a dictionary server.Dictionary server helps  to maintain dictionary values independent of the data loading and there by avoids reading the same input data multiples times.This configuration determines the number of concurrent dictionary generation or request that needs to be served by the dictionary server.**NOTE:** This configuration takes effect when ***carbon.options.single.pass*** is configured as true.Please refer to *carbon.options.single.pass*to understand how dictionary server optimizes data loading. |
+| enable.unsafe.sort | true | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations.This configuration enables to use unsafe functions in CarbonData.**NOTE:** For operations like data loading, which generates more short lived Java objects, Java GC can be a bottle neck.Using unsafe can overcome the GC overhead and improve the overall performance. |
+| enable.offheap.sort | true | CarbonData supports storing data in off-heap memory for certain operations during data loading and query.This helps to avoid the Java GC and thereby improve the overall performance.This configuration enables using off-heap memory for sorting of data during data loading.**NOTE:**  ***enable.unsafe.sort*** configuration needs to be configured to true for using off-heap |
+| enable.inmemory.merge.sort | false | CarbonData sorts and writes data to intermediate files to limit the memory usage.These intermediate files needs to be sorted again using merge sort before writing to the final carbondata file.Performing merge sort in memory would increase the sorting performance at the cost of increased memory footprint. This Configuration specifies to do in-memory merge sort or to do file based merge sort. |
+| carbon.load.sort.scope | LOCAL_SORT | CarbonData can support various sorting options to match the balance between load and query performance.LOCAL_SORT:All the data given to an executor in the single load is fully sorted and written to carondata files.Data loading performance is reduced a little as the entire data needs to be sorted in the executor.BATCH_SORT:Sorts the data in batches of configured size and writes to carbondata files.Data loading performance increases as the entire data need not be sorted.But query performance will get reduced due to false positives in block pruning and also due to more number of carbondata files written.Due to more number of carbondata files, if identified blocks > cluster parallelism, query performance and concurrency will get reduced.GLOBAL SORT:Entire data in the data load is fully sorted and written to carbondata files.Data loading perfromance would get reduced as the entire data needs to be sorted.But the query performance increases signific
 antly due to very less false positives and concurrency is also improved.**NOTE:** when BATCH_SORTis configured, it is recommended to keep ***carbon.load.batch.sort.size.inmb*** > ***carbon.blockletgroup.size.in.mb*** |
+| carbon.load.batch.sort.size.inmb | 0 | When  ***carbon.load.sort.scope*** is configured as ***BATCH_SORT***,This configuration needs to be added to specify the batch size for sorting and writing to carbondata files.**NOTE:** It is recommended to keep the value around 45% of ***carbon.sort.storage.inmemory.size.inmb*** to avoid spill to disk.Also it is recommended to keep the value higher than ***carbon.blockletgroup.size.in.mb***. Refer to *carbon.load.sort.scope* for more information on sort options and the advantages/disadvantges of each option. |
+| carbon.dictionary.server.port | 2030 | 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.Single pass loading can be enabled using the option ***carbon.options.single.pass***.When this option is specified, a dictionary server will be internally started to handle the dictionary generation and query requests.This configuration specifies the port on which the server need to listen for incoming requests.Port value ranges between 0-65535 |
+| carbon.merge.sort.prefetch | true | CarbonData writes every ***carbon.sort.size*** number of records to intermediate temp files during data loading to ensure memory footprint is within limits.These intermediate temp files will have to be sorted using merge sort before writing into CarbonData format.This configuration enables pre fetching of data from these temp files in order to optimize IO and speed up data loading process. |
+| carbon.loading.prefetch | false | CarbonData uses univocity parser to read csv files.This configuration is used to inform the parser whether it can prefetch the data from csv files to speed up the reading.**NOTE:** Enabling prefetch improves the data loading performance, but needs higher memory to keep more records which are read ahead from disk. |
+| carbon.prefetch.buffersize | 1000 | When the configuration ***carbon.merge.sort.prefetch*** is configured to true, we need to set the number of records that can be prefetched.This configuration is used specify the number of records to be prefetched.**NOTE: **Configuring more number of records to be prefetched increases memory footprint as more records will have to be kept in memory. |
+| load_min_size_inmb | 256 | This configuration is used along with ***carbon.load.min.size.enabled***.This determines the minimum size of input files to be considered for distribution among executors while data loading.**NOTE:** Refer to ***carbon.load.min.size.enabled*** for understanding when this configuration needs to be used and its advantages and disadvantages. |
+| carbon.load.sortmemory.spill.percentage | 0 | During data loading, some data pages are kept in memory upto memory configured in ***carbon.sort.storage.inmemory.size.inmb*** beyond which they are spilled to disk as intermediate temporary sort files.This configuration determines after what percentage data needs to be spilled to disk.**NOTE:** Without this configuration, when the data pages occupy upto configured memory, new data pages would be dumped to disk and old pages are still maintained in disk. |
+| carbon.load.directWriteHdfs.enabled | false | During data load all the carbondata files are written to local disk and finally copied to the target location in HDFS.Enabling this parameter will make carrbondata files to be written directly onto target HDFS location bypassing the local disk.**NOTE:** Writing directly to HDFS saves local disk IO(once for writing the files and again for copying to HDFS) there by improving the performance.But the drawback is when data loading fails or the application crashes, unwanted carbondata files will remain in the target HDFS location until it is cleared during next data load or by running *CLEAN FILES* DDL command |
+| carbon.options.serialization.null.format | \N | Based on the business scenarios, some columns might need to be loaded with null values.As null value cannot be written in csv files, some special characters might be adopted to specify null values.This configuration can be used to specify the null values format in the data being loaded. |
+| carbon.sort.storage.inmemory.size.inmb | 512 | CarbonData writes every ***carbon.sort.size*** number of records to intermediate temp files during data loading to ensure memory footprint is within limits.When ***enable.unsafe.sort*** configuration is enabled, instead of using ***carbon.sort.size*** which is based on rows count, size occupied in memory is used to determine when to flush data pages to intermediate temp files.This configuration determines the memory to be used for storing data pages in memory.**NOTE:** Configuring a higher values ensures more data is maintained in memory and hence increases data loading performance due to reduced or no IO.Based on the memory availability in the nodes of the cluster, configure the values accordingly. |
+
+## Compaction Configuration
 
- 
-* **Query Configuration**
+| Parameter | Default Value | Description |
+|-----------------------------------------------|---------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| carbon.number.of.cores.while.compacting | 2 | Number of cores to be used while compacting data.This also determines the number of threads to be used to read carbondata files in parallel. |
+| carbon.compaction.level.threshold | 4, 3 | Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance.This configuration is for minor compaction which decides how many segments to be merged. Configuration is of the form (x,y). Compaction will be triggered for every x segments and form a single level 1 compacted segment.When the number of compacted level 1 segments reach y, compaction will be triggered again to merge them to form a single level 2 segment. For example: If it is set as 2, 3 then minor compaction will be triggered for every 2 segments. 3 is the number of level 1 compacted segments which is further compacted to new segment.**NOTE:** When ***carbon.enable.auto.load.merge*** is **true**, Configuring higher values cause overall data loading time to increase as compaction will be triggered after data loading is complete but status is not returned till compaction is comp
 lete. But compacting more number of segments can increase query performance.Hence optimal values needs to be configured based on the business scenario.Valid values are bwteen 0 to 100. |
+| carbon.major.compaction.size | 1024 | To improve query performance and All the segments can be merged and compacted to a single segment upto configured size.This Major compaction size can be configured using this parameter. Sum of the segments which is below this threshold will be merged. This value is expressed in MB. |
+| carbon.horizontal.compaction.enable | true | CarbonData supports DELETE/UPDATE functionality by creating delta data files for existing carbondata files.These delta files would grow as more number of DELETE/UPDATE operations are performed.Compaction of these delta files are termed as horizontal compaction.This configuration is used to turn ON/OFF horizontal compaction. After every DELETE and UPDATE statement, horizontal compaction may occur in case the delta (DELETE/ UPDATE) files becomes more than specified threshold.**NOTE: **Having many delta files will reduce the query performance as scan has to happen on all these files before the final state of data can be decided.Hence it is advisable to keep horizontal compaction enabled and configure reasonable values to ***carbon.horizontal.UPDATE.compaction.threshold*** and ***carbon.horizontal.DELETE.compaction.threshold*** |
+| carbon.horizontal.update.compaction.threshold | 1 | This configuration specifies the threshold limit on number of UPDATE delta files within a segment. In case the number of delta files goes beyond the threshold, the UPDATE delta files within the segment becomes eligible for horizontal compaction and are compacted into single UPDATE delta file.Values range between 1 to 10000. |
+| carbon.horizontal.delete.compaction.threshold | 1 | This configuration specifies the threshold limit on number of DELETE delta files within a block of a segment. In case the number of delta files goes beyond the threshold, the DELETE delta files for the particular block of the segment becomes eligible for horizontal compaction and are compacted into single DELETE delta file.Values range between 1 to 10000. |
+| carbon.update.segment.parallelism | 1 | CarbonData processes the UPDATE operations by grouping records belonging to a segment into a single executor task.When the amount of data to be updated is more, this behavior causes problems like restarting of executor due to low memory and data-spill related errors.This property specifies the parallelism for each segment during update.**NOTE:** It is recommended to set this value to a multiple of the number of executors for balance.Values range between 1 to 1000. |
+| carbon.numberof.preserve.segments | 0 | If the user wants to preserve some number of segments from being compacted then he can set this configuration. Example: carbon.numberof.preserve.segments = 2 then 2 latest segments will always be excluded from the compaction. No segments will be preserved by default.**NOTE:** This configuration is useful when the chances of input data can be wrong due to environment scenarios.Preserving some of the latest segments from being compacted can help to easily delete the wrongly loaded segments.Once compacted,it becomes more difficult to determine the exact data to be deleted(except when data is incrementing according to time) |
+| carbon.allowed.compaction.days | 0 | This configuration is used to control on the number of recent segments that needs to be compacted, ignoring the older ones.This congifuration is in days.For Example: If the configuration is 2, then the segments which are loaded in the time frame of past 2 days only will get merged. Segments which are loaded earlier than 2 days will not be merged. This configuration is disabled by default.**NOTE:** This configuration is useful when a bulk of history data is loaded into the carbondata.Query on this data is less frequent.In such cases involving these segments also into compacation will affect the resource consumption, increases overall compaction time. |
+| carbon.enable.auto.load.merge | false | Compaction can be automatically triggered once data load completes.This ensures that the segments are merged in time and thus query times doesnt increase with increase in segments.This configuration enables to do compaction along with data loading.**NOTE: **Compaction will be triggered once the data load completes.But the status of data load wait till the compaction is completed.Hence it might look like data loading time has increased, but thats not the case.Moreover failure of compaction will not affect the data loading status.If data load had completed successfully, the status would be updated and segments are committed.However, failure while data loading, will not trigger compaction and error is returned immediately. |
+| carbon.enable.page.level.reader.in.compaction|true|Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory.**NOTE:** Please refer to [file-structure-of-carbondata](./file-structure-of-carbondata.md ) to understand the storage format of CarbonData and concepts of pages.|
+| carbon.concurrent.compaction | true | Compaction of different tables can be executed concurrently.This configuration determines whether to compact all qualifying tables in parallel or not.**NOTE: **Compacting concurrently is a resource demanding operation and needs more resouces there by affecting the query performance also.This configuration is **deprecated** and might be removed in future releases. |
+| carbon.compaction.prefetch.enable | false | Compaction operation is similar to Query + data load where in data from qualifying segments are queried and data loading performed to generate a new single segment.This configuration determines whether to query ahead data from segments and feed it for data loading.**NOTE: **This configuration is disabled by default as it needs extra resources for querying ahead extra data.Based on the memory availability on the cluster, user can enable it to improve compaction performance. |
+| carbon.merge.index.in.segment | true | Each CarbonData file has a companion CarbonIndex file which maintains the metadata about the data.These CarbonIndex files are read and loaded into driver and is used subsequently for pruning of data during queries.These CarbonIndex files are very small in size(few KB) and are many.Reading many small files from HDFS is not efficient and leads to slow IO performance.Hence these CarbonIndex files belonging to a segment can be combined into  a single file and read once there by increasing the IO throughput.This configuration enables to merge all the CarbonIndex files into a single MergeIndex file upon data loading completion.**NOTE:** Reading a single big file is more efficient in HDFS and IO throughput is very high.Due to this the time needed to load the index files into memory when query is received for the first time on that table is significantly reduced and there by significantly reduces the delay in serving the first query. |
+
+## Query Configuration
 
 | Parameter | Default Value | Description |
-|--------------------------|---------------|-----------------------------------------------------------------------------------------------|
+|--------------------------------------|---------------|---------------------------------------------------|
+| carbon.max.driver.lru.cache.size | -1 | Maximum memory **(in MB)** upto which the driver process can cache the data (BTree and dictionary values). Beyond this, least recently used data will be removed from cache before loading new set of values.Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.**NOTE:** Minimum number of entries that needs to be removed from cache in order to load the new set of data is determined and unloaded.ie.,for example if 3 cache entries qualify for pre-emption, out of these, those entries that free up more cache memory is removed prior to others. |
+| carbon.max.executor.lru.cache.size | -1 | Maximum memory **(in MB)** upto which the executor process can cache the data (BTree and reverse dictionary values).Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.**NOTE:** If this parameter is not configured, then the value of ***carbon.max.driver.lru.cache.size*** will be used. |
 | max.query.execution.time | 60 | Maximum time allowed for one query to be executed. The value is in minutes. |
-| carbon.enableMinMax | true | Min max is feature added to enhance query performance. To disable this feature, set it false. |
-| carbon.dynamicallocation.schedulertimeout | 5 | Specifies the maximum time (unit in seconds) the scheduler can wait for executor to be active. Minimum value is 5 sec and maximum value is 15 sec. |
-| carbon.scheduler.minregisteredresourcesratio | 0.8 | Specifies the minimum resource (executor) ratio needed for starting the block distribution. The default value is 0.8, which indicates 80% of the requested resource is allocated for starting block distribution.  The minimum value is 0.1 min and the maximum value is 1.0. | 
+| carbon.enableMinMax | true | CarbonData maintains the metadata which enables to prune unnecessary files from being scanned as per the query conditions.To achieve pruning, Min,Max of each column is maintined.Based on the filter condition in the query, certain data can be skipped from scanning by matching the filter value against the min,max values of the column(s) present in that carbondata file.This pruing enhances query performance significantly. |
+| carbon.dynamicallocation.schedulertimeout | 5 | CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData.To determine the number of tasks that can be scheduled, knowing the count of active executors is necessary.When dynamic allocation is enabled on a YARN based spark cluster,execuor processes are shutdown if no request is received for a particular amount of time.The executors are brought up when the requet is received again.This configuration specifies the maximum time (unit in seconds) the carbon scheduler can wait for executor to be active. Minimum value is 5 sec and maximum value is 15 sec.**NOTE: **Waiting for longer time leads to slow query response time.Moreover it might be possible that YARN is not able to start the executors and waiting is not beneficial. |
+| carbon.scheduler.minregisteredresourcesratio | 0.8 | Specifies the minimum resource (executor) ratio needed for starting the block distribution. The default value is 0.8, which indicates 80% of the requested resource is allocated for starting block distribution.  The minimum value is 0.1 min and the maximum value is 1.0. |
 | carbon.search.enabled (Alpha Feature) | false | If set to true, it will use CarbonReader to do distributed scan directly instead of using compute framework like spark, thus avoiding limitation of compute framework like SQL optimizer and task scheduling overhead. |
-
-* **Global Dictionary Configurations**
-  
-| Parameter | Default Value | Description |
-|---------------------------------------|---------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| carbon.cutOffTimestamp |  | Sets the start date for calculating the timestamp. Java counts the number of milliseconds from start of "1970-01-01 00:00:00". This property is used to customize the start of position. For example "2000-01-01 00:00:00". The date must be in the form "carbon.timestamp.format". |
-| carbon.timegranularity | SECOND | The property used to set the data granularity level DAY, HOUR, MINUTE, or SECOND. |
-  
-##  Spark Configuration
- <b><p align="center">Spark Configuration Reference in spark-defaults.conf</p></b>
- 
+| carbon.search.query.timeout | 10s | Time within which the result is expected from the workers;beyond which the query is terminated |
+| carbon.search.scan.thread | num of cores available in worker node | Number of cores to be used in each worker for performing scan. |
+| carbon.search.master.port | 10020 | Port on which the search master listens for incoming query requests |
+| carbon.search.worker.port | 10021 | Port on which search master communicates with the workers. |
+| carbon.search.worker.workload.limit | 10 * *carbon.search.scan.thread* | Maximum number of active requests that can be sent to a worker.Beyond which the request needs to be rescheduled for later time or to a different worker. |
+| carbon.detail.batch.size | 100 | The buffer size to store records, returned from the block scan. In limit scenario this parameter is very important. For example your query limit is 1000. But if we set this value to 3000 that means we get 3000 records from scan but spark will only take 1000 rows. So the 2000 remaining are useless. In one Finance test case after we set it to 100, in the limit 1000 scenario the performance increase about 2 times in comparison to if we set this value to 12000. |
+| carbon.enable.vector.reader | true | Spark added vector processing to optimize cpu cache miss and there by increase the query performance.This configuration enables to fetch data as columnar batch of size 4*1024 rows instead of fetching data row by row and provide it to spark so that there is improvement in  select queries performance. |
+| carbon.task.distribution | block | CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData.Each of these task distribution suggestions has its own advantages and disadvantages.Based on the customer use case, appropriate task distribution can be configured.**block**: Setting this value will launch one task per block. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. **custom**: Setting this value will group the blocks and distribute it uniformly to the available resources in the cluster. This enhances the query performance but not suggested in case of concurrent queries and queries having big shuffling scenarios. **blocklet**: Setting this value will launch one task per blocklet. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. **merge_small_files**: S
 etting this value will merge all the small carbondata files upto a bigger size configured by ***spark.sql.files.maxPartitionBytes*** (128 MB is the default value,it is configurable) during querying. The small carbondata files are combined to a map task to reduce the number of read task. This enhances the performance. |
+| carbon.custom.block.distribution | false | CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData.When this configuration is true, CarbonData would distribute the available blocks to be scanned among the available number of cores.For Example:If there are 10 blocks to be scanned and only 3 tasks can be run(only 3 executor cores available in the cluster), CarbonData would combine blocks as 4,3,3 and give it to 3 tasks to run.**NOTE:** When this configuration is false, as per the ***carbon.task.distribution*** configuration, each block/blocklet would be given to each task. |
+| enable.query.statistics | false | CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues.This configuration when made ***true*** would log additional query statistics information to more accurately locate the issues being debugged.**NOTE:** Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time.It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately.Also extensive logging is an increased IO operation and hence over all query performance might get reduced.Therefore it is recommened to enable this configuration only for the duration of debugging. |
+| enable.unsafe.in.query.processing | true | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations.This configuration enables to use unsafe functions in CarbonData while scanning the  data during query. |
+| carbon.query.validate.directqueryondatamap | true | CarbonData supports creating pre-aggregate table datamaps as an independent tables.For some debugging purposes, it might be required to directly query from such datamap tables.This configuration allows to query on such datamaps. |
+| carbon.heap.memory.pooling.threshold.bytes | 1048576 | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations.Using unsafe, memory can be allocated on Java Heap or off heap.This configuration controlls the allocation mechanism on Java HEAP.If the heap memory allocations of the given size is greater or equal than this value,it should go through the pooling mechanism.But if set this size to -1, it should not go through the pooling mechanism.Default value is 1048576(1MB, the same as Spark).Value to be specified in bytes. |
+
+## Data Mutation Configuration
 | Parameter | Default Value | Description |
-|----------------------------------------|--------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| spark.driver.memory | 1g | Amount of memory to be used by the driver process. |
-| spark.executor.memory | 1g | Amount of memory to be used per executor process. |
+|--------------------------------------|---------------|---------------------------------------------------|
+| carbon.insert.persist.enable | false | CarbonData does loading in 2 major steps.1st step reads from the input source and generates the dictionary values.2nd step reads from the source again and encodes the data with the dictionary values, perform index calculations and writes in CarbonData format. Suppose we are loading the CarbonData table using another table as source(using insert into) and the source table is being loaded in parallel, there can be cases where some data got inserted into the source table after CarbonData generated for the target table in which case some new records which does not have dictionary values generated gets read leading to inconsistency. To avoid this condition we can persist the dataset of RDD/dataframe into MEMORY_AND_DISK(default value) and perform insert into operation. This ensures the data read from source table is cached and is not read again from the source there by ensuring consistency between dictionary generation and writing to CarbonData fo
 rmat steps. By default this value is false as concurrent loading into source table is not the scenario majority of the times.**NOTE:** This configuration can reduce the insert into execution time as data need not be re read; but increases the memory foot print. |
+| carbon.insert.storage.level | MEMORY_AND_DISK | Storage level to persist dataset of a RDD/dataframe.Applicable when ***carbon.insert.persist.enable*** is **true**, if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. [See detail](http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence). |
+| carbon.update.persist.enable | true | Configuration to enable the dataset of RDD/dataframe to persist data. Enabling this will reduce the execution time of UPDATE operation. |
+| carbon.update.storage.level | MEMORY_AND_DISK | Storage level to persist dataset of a RDD/dataframe.Applicable when ***carbon.update.persist.enable*** is **true**, if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. [See detail](http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence). |
+
 
 ##  Dynamic Configuration In CarbonData Using SET-RESET
 
@@ -208,16 +191,24 @@ RESET
 
 <b><p align="center">Dynamically Configurable Properties of CarbonData</p></b>
 
-| Properties                               | Description                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                              |
-|------------------------------------------|------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| carbon.options.bad.records.logger.enable | To enable or disable bad record logger.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                  |
-| carbon.options.bad.records.action        | This property can have four types of actions for bad records FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found.                                                                                                                                                          |
-| carbon.options.is.empty.data.bad.record  | If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                          |
-| carbon.options.batch.sort.size.inmb      | Size of batch data to keep in memory, as a thumb rule it supposed to be less than 45% of sort.inmemory.size.inmb otherwise it may spill intermediate data to disk.                                                                                                                                                                                                                                                                                                                                                                                                                       |
-| carbon.options.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.                                                                                                                                                                                                               |
-| carbon.options.bad.record.path           | Specifies the HDFS path where bad records needs to be stored.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                            |
-| carbon.custom.block.distribution         | Specifies whether to use the Spark or Carbon block distribution feature.                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                                 |
-| enable.unsafe.sort                       | Specifies whether to use unsafe sort during data loading. Unsafe sort reduces the garbage collection during data load operation, resulting in better performance.                                                                                                                                                                                                                                                                                                                                                                                                                        |
+
+| Properties                                | Description                                                  |
+| ----------------------------------------- | ------------------------------------------------------------ |
+| carbon.options.bad.records.logger.enable  | CarbonData can identify the records that are not conformant to schema and isolate them as bad records.Enabling this configuration will make CarbonData to log such bad records.**NOTE:** If the input data contains many bad records, logging them will slow down the over all data loading throughput.The data load operation status would depend on the configuration in ***carbon.bad.records.action***. |
+| carbon.options.bad.records.logger.enable  | To enable or disable bad record logger.                      |
+| carbon.options.bad.records.action         | This property can have four types of actions for bad records FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found. |
+| carbon.options.is.empty.data.bad.record   | If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa. |
+| carbon.options.batch.sort.size.inmb       | Size of batch data to keep in memory, as a thumb rule it supposed to be less than 45% of sort.inmemory.size.inmb otherwise it may spill intermediate data to disk. |
+| carbon.options.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.**NOTE:** Enabling this starts a new dictionary server to handle dictionary generation requests during data loading.Without this option, the input csv files will have to read twice.Once while dictionary generation and persisting to the dictionary files.second when the data loading need to convert the input data into carbondata format.Enabling this optimizes the optimizes to read the input data only once there by reducing IO and hence over all data loading time.If concurrent data loading needs to be supported, consider tuning ***dictionary.worker.threads***.Port on which the dictiona
 ry server need to listen on can be configured using the configuration ***carbon.dictionary.server.port***. |
+| carbon.options.bad.record.path            | Specifies the HDFS path where bad records needs to be stored. |
+| carbon.custom.block.distribution          | Specifies whether to use the Spark or Carbon block distribution feature.**NOTE: **Refer to [Query Configuration](#query-configuration)#carbon.custom.block.distribution for more details on CarbonData scheduler. |
+| enable.unsafe.sort                        | Specifies whether to use unsafe sort during data loading. Unsafe sort reduces the garbage collection during data load operation, resulting in better performance. |
+| carbon.options.dateformat                 | Specifies the data format of the date columns in the data being loaded |
+| carbon.options.timestampformat            | Specifies the timestamp format of the time stamp columns in the data being loaded |
+| carbon.options.sort.scope                 | Specifies how the current data load should be sorted with.**NOTE: **Refer to [Data Loading Configuration](#data-loading-configuration)#carbon.sort.scope for detailed information. |
+| carbon.options.global.sort.partitions     |                                                              |
+| carbon.options.serialization.null.format  | Default Null value representation in the data being loaded.**NOTE:** Refer to [Data Loading Configuration](#data-loading-configuration)#carbon.options.serialization.null.format for detailed information. |
+| carbon.query.directQueryOnDataMap.enabled | Specifies whether datamap can be queried directly.This is useful for debugging purposes.**NOTE: **Refer to [Query Configuration](#query-configuration)#carbon.query.validate.directqueryondatamap for detailed information. |
 
 **Examples:**
 
@@ -244,3 +235,16 @@ RESET
 * Success will be recorded in the driver log.
 
 * Failure will be displayed in the UI.
+
+
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file


[27/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/MDFileConverter.scala
----------------------------------------------------------------------
diff --git a/src/main/scala/MDFileConverter.scala b/src/main/scala/MDFileConverter.scala
index 79d1d27..9c5e393 100644
--- a/src/main/scala/MDFileConverter.scala
+++ b/src/main/scala/MDFileConverter.scala
@@ -12,6 +12,7 @@ class MDFileConverter @Inject()(fileService: FileService, confService: ConfServi
   val outputFileExtension: String = ".html"
   val headerContent: String = fileService.readFromFile(confService.readString("headerPath"))
   val footerContent: String = fileService.readFromFile(confService.readString("footerPath"))
+  val scriptsPath: String = confService.readString("scriptsPath")
   val location: String = confService.readString("outputFileLocation")
   val fileReadObject: MdFileHandler = new MdFileHandler(confService, fileService)
   val failMessage: String = "failure"
@@ -84,8 +85,11 @@ class MDFileConverter @Inject()(fileService: FileService, confService: ConfServi
     getFileData match {
       case Some(data: String) => val fileData = fileReadObject.convertMdExtensions(data)
         logger.info(s"Begin writing [ $fileName outputFileExtension ] at $location")
+
+        val scriptsContent: String = fileService.readFromFile(scriptsPath + fileName)
+
         val statusHtmlFile = fileService
-          .writeToFile(outputPath, headerContent + fileData + footerContent)
+          .writeToFile(outputPath, headerContent + fileData + scriptsContent + footerContent)
         saveMdFilesForPDF(dirStatus, fileURLContent, fileName)
         if (imageFileList.contains(fileName)) {
           saveMdFilesForPDF(dirStatus, changeImageLink(fileURLContent), fileName)

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/MdFileHandler.scala
----------------------------------------------------------------------
diff --git a/src/main/scala/MdFileHandler.scala b/src/main/scala/MdFileHandler.scala
index 6fae74e..148a280 100644
--- a/src/main/scala/MdFileHandler.scala
+++ b/src/main/scala/MdFileHandler.scala
@@ -18,13 +18,17 @@ class MdFileHandler @Inject()(confService: ConfService, fileService: FileService
   def convertMdExtensions(input: String): String = {
     val modifyContentPattern: Regex = new Regex("id=\"user-content-")
     val modifyMdPattern: Regex = new Regex(".md")
+    val modifyDatamapPattern: Regex = new Regex("./datamap/")
     val modifyImagePattern: Regex = new Regex("<img src=\"../docs")
     val modifyHttpsFileLink: Regex ="""(<a href=\"https)://([a-zA-Z0-9-/.]+)(\")""".r
     val modifyHttpFileLink: Regex ="""(<a href=\"http)://([a-zA-Z0-9-/.]+)(\")""".r
     val replacingImageContent: String = "<img src=\"https://github.com/apache/carbondata/blob/master/docs"
     val contentAfterRemovingUserContent: String = modifyContentPattern replaceAllIn(input, "id=\"")
     val contentAfterReplacingId: String = modifyMdPattern replaceAllIn(contentAfterRemovingUserContent, ".html")
-    val contentAfterReplacingImage: String = modifyImagePattern replaceAllIn(contentAfterReplacingId,replacingImageContent)
+
+    val contentAfterReplacingDatamap: String = modifyDatamapPattern replaceAllIn(contentAfterReplacingId, "./")
+
+    val contentAfterReplacingImage: String = modifyImagePattern replaceAllIn(contentAfterReplacingDatamap,replacingImageContent)
     val contentAfterReplacingHttpsFileLink: String = modifyHttpsFileLink replaceAllIn(contentAfterReplacingImage, "$1://$2$3 target=_blank")
     val contentAfterReplacingFileLink: String = modifyHttpFileLink replaceAllIn(contentAfterReplacingHttpsFileLink, "$1://$2$3 target=_blank")
     contentAfterReplacingFileLink

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/html/header.html
----------------------------------------------------------------------
diff --git a/src/main/scala/html/header.html b/src/main/scala/html/header.html
index 28cab9c..217a9f4 100644
--- a/src/main/scala/html/header.html
+++ b/src/main/scala/html/header.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/bloomfilter-datamap-guide
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/bloomfilter-datamap-guide b/src/main/scala/scripts/bloomfilter-datamap-guide
new file mode 100644
index 0000000..8834f00
--- /dev/null
+++ b/src/main/scala/scripts/bloomfilter-datamap-guide
@@ -0,0 +1,11 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/configuration-parameters
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/configuration-parameters b/src/main/scala/scripts/configuration-parameters
new file mode 100644
index 0000000..3d168cd
--- /dev/null
+++ b/src/main/scala/scripts/configuration-parameters
@@ -0,0 +1,11 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/datamap-developer-guide
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/datamap-developer-guide b/src/main/scala/scripts/datamap-developer-guide
new file mode 100644
index 0000000..3d168cd
--- /dev/null
+++ b/src/main/scala/scripts/datamap-developer-guide
@@ -0,0 +1,11 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/datamap-management
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/datamap-management b/src/main/scala/scripts/datamap-management
new file mode 100644
index 0000000..8834f00
--- /dev/null
+++ b/src/main/scala/scripts/datamap-management
@@ -0,0 +1,11 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/ddl-of-carbondata
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/ddl-of-carbondata b/src/main/scala/scripts/ddl-of-carbondata
new file mode 100644
index 0000000..3d168cd
--- /dev/null
+++ b/src/main/scala/scripts/ddl-of-carbondata
@@ -0,0 +1,11 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/dml-of-carbondata
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/dml-of-carbondata b/src/main/scala/scripts/dml-of-carbondata
new file mode 100644
index 0000000..3d168cd
--- /dev/null
+++ b/src/main/scala/scripts/dml-of-carbondata
@@ -0,0 +1,11 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/faq
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/faq b/src/main/scala/scripts/faq
new file mode 100644
index 0000000..10ea449
--- /dev/null
+++ b/src/main/scala/scripts/faq
@@ -0,0 +1,4 @@
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__faq').addClass('selected'); });
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/file-structure-of-carbondata
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/file-structure-of-carbondata b/src/main/scala/scripts/file-structure-of-carbondata
new file mode 100644
index 0000000..3d168cd
--- /dev/null
+++ b/src/main/scala/scripts/file-structure-of-carbondata
@@ -0,0 +1,11 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/how-to-contribute-to-apache-carbondata
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/how-to-contribute-to-apache-carbondata b/src/main/scala/scripts/how-to-contribute-to-apache-carbondata
new file mode 100644
index 0000000..c92b67b
--- /dev/null
+++ b/src/main/scala/scripts/how-to-contribute-to-apache-carbondata
@@ -0,0 +1,4 @@
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__contri').addClass('selected'); });
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/introduction
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/introduction b/src/main/scala/scripts/introduction
new file mode 100644
index 0000000..c523ce7
--- /dev/null
+++ b/src/main/scala/scripts/introduction
@@ -0,0 +1,4 @@
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__intro').addClass('selected'); });
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/language-manual
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/language-manual b/src/main/scala/scripts/language-manual
new file mode 100644
index 0000000..3d168cd
--- /dev/null
+++ b/src/main/scala/scripts/language-manual
@@ -0,0 +1,11 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/lucene-datamap-guide
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/lucene-datamap-guide b/src/main/scala/scripts/lucene-datamap-guide
new file mode 100644
index 0000000..8834f00
--- /dev/null
+++ b/src/main/scala/scripts/lucene-datamap-guide
@@ -0,0 +1,11 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/performance-tuning
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/performance-tuning b/src/main/scala/scripts/performance-tuning
new file mode 100644
index 0000000..f884fec
--- /dev/null
+++ b/src/main/scala/scripts/performance-tuning
@@ -0,0 +1,4 @@
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__perf').addClass('selected'); });
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/preaggregate-datamap-guide
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/preaggregate-datamap-guide b/src/main/scala/scripts/preaggregate-datamap-guide
new file mode 100644
index 0000000..8834f00
--- /dev/null
+++ b/src/main/scala/scripts/preaggregate-datamap-guide
@@ -0,0 +1,11 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/quick-start-guide
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/quick-start-guide b/src/main/scala/scripts/quick-start-guide
new file mode 100644
index 0000000..3bfec2b
--- /dev/null
+++ b/src/main/scala/scripts/quick-start-guide
@@ -0,0 +1,4 @@
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__quickstart').addClass('selected'); });
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/release-guide
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/release-guide b/src/main/scala/scripts/release-guide
new file mode 100644
index 0000000..3bfe498
--- /dev/null
+++ b/src/main/scala/scripts/release-guide
@@ -0,0 +1,4 @@
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__release').addClass('selected'); });
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/s3-guide
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/s3-guide b/src/main/scala/scripts/s3-guide
new file mode 100644
index 0000000..ee9dc1f
--- /dev/null
+++ b/src/main/scala/scripts/s3-guide
@@ -0,0 +1,4 @@
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__s3').addClass('selected'); });
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/sdk-guide
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/sdk-guide b/src/main/scala/scripts/sdk-guide
new file mode 100644
index 0000000..700185a
--- /dev/null
+++ b/src/main/scala/scripts/sdk-guide
@@ -0,0 +1,4 @@
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__api').addClass('selected'); });
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/segment-management-on-carbondata
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/segment-management-on-carbondata b/src/main/scala/scripts/segment-management-on-carbondata
new file mode 100644
index 0000000..48bd74b
--- /dev/null
+++ b/src/main/scala/scripts/segment-management-on-carbondata
@@ -0,0 +1,10 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/streaming-guide
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/streaming-guide b/src/main/scala/scripts/streaming-guide
new file mode 100644
index 0000000..3d168cd
--- /dev/null
+++ b/src/main/scala/scripts/streaming-guide
@@ -0,0 +1,11 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/supported-data-types-in-carbondata
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/supported-data-types-in-carbondata b/src/main/scala/scripts/supported-data-types-in-carbondata
new file mode 100644
index 0000000..3d168cd
--- /dev/null
+++ b/src/main/scala/scripts/supported-data-types-in-carbondata
@@ -0,0 +1,11 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/timeseries-datamap-guide
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/timeseries-datamap-guide b/src/main/scala/scripts/timeseries-datamap-guide
new file mode 100644
index 0000000..8834f00
--- /dev/null
+++ b/src/main/scala/scripts/timeseries-datamap-guide
@@ -0,0 +1,11 @@
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__datamap').addClass('selected');
+  
+  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
+    // Display datamap subnav items
+    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/scala/scripts/usecases
----------------------------------------------------------------------
diff --git a/src/main/scala/scripts/usecases b/src/main/scala/scripts/usecases
new file mode 100644
index 0000000..4e123f8
--- /dev/null
+++ b/src/main/scala/scripts/usecases
@@ -0,0 +1,4 @@
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__uses').addClass('selected'); });
+</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/bloomfilter-datamap-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/bloomfilter-datamap-guide.html b/src/main/webapp/bloomfilter-datamap-guide.html
index 54c14fe..8030599 100644
--- a/src/main/webapp/bloomfilter-datamap-guide.html
+++ b/src/main/webapp/bloomfilter-datamap-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -260,7 +260,7 @@ User can create BloomFilter datamap on specified columns with specified BloomFil
   age int,
   city string,
   country string)
-STORED BY 'carbondata'
+STORED AS carbondata
 TBLPROPERTIES('SORT_COLUMNS'='id')
 </code></pre>
 <p>In the above example, <code>id</code> and <code>name</code> are high cardinality columns
@@ -366,8 +366,7 @@ $(function() {
     $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -382,4 +381,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/configuration-parameters.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/configuration-parameters.html b/src/main/webapp/configuration-parameters.html
index ba73b0d..ab89576 100644
--- a/src/main/webapp/configuration-parameters.html
+++ b/src/main/webapp/configuration-parameters.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -212,7 +212,7 @@
                                     <div>
 <h1>
 <a id="configuring-carbondata" class="anchor" href="#configuring-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configuring CarbonData</h1>
-<p>This guide explains the configurations that can be used to tune CarbonData to achieve better performance.Some of the properties can be set dynamically and are explained in the section Dynamic Configuration In CarbonData Using SET-RESET.Most of the properties that control the internal settings have reasonable default values.They are listed along with the properties along with explanation.</p>
+<p>This guide explains the configurations that can be used to tune CarbonData to achieve better performance.Most of the properties that control the internal settings have reasonable default values.They are listed along with the properties along with explanation.</p>
 <ul>
 <li><a href="#system-configuration">System Configuration</a></li>
 <li><a href="#data-loading-configuration">Data Loading Configuration</a></li>
@@ -364,7 +364,7 @@
 <tr>
 <td>carbon.blockletgroup.size.in.mb</td>
 <td>64</td>
-<td>Please refer to <a href="./file-structure-of-carbondata.html">file-structure-of-carbondata</a> to understand the storage format of CarbonData.The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of each blocklet group. Higher value results in better sequential IO access.The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB).<strong>NOTE:</strong> Configuring a higher value might lead to poor performance as an entire blocklet group will have to read into memory before processing.For filter queries with limit, it is <strong>not advisable</strong> to have a bigger blocklet size.For Aggregation queries which need to return more number of rows,bigger blocklet size is advisable.</td>
+<td>Please refer to <a href="./file-structure-of-carbondata.html#carbondata-file-format">file-structure-of-carbondata</a> to understand the storage format of CarbonData.The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of each blocklet group. Higher value results in better sequential IO access.The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB).<strong>NOTE:</strong> Configuring a higher value might lead to poor performance as an entire blocklet group will have to read into memory before processing.For filter queries with limit, it is <strong>not advisable</strong> to have a bigger blocklet size.For Aggregation queries which need to return more number of rows,bigger blocklet size is advisable.</td>
 </tr>
 <tr>
 <td>carbon.sort.file.write.buffer.size</td>
@@ -420,7 +420,7 @@
 <tr>
 <td>carbon.use.local.dir</td>
 <td>false</td>
-<td>CarbonData during data loading, writes files to local temp directories before copying the files to HDFS.This configuration is used to specify whether CarbonData can write locally to tmp directory of the container or to the YARN application directory.</td>
+<td>CarbonData,during data loading, writes files to local temp directories before copying the files to HDFS.This configuration is used to specify whether CarbonData can write locally to tmp directory of the container or to the YARN application directory.</td>
 </tr>
 <tr>
 <td>carbon.use.multiple.temp.dir</td>
@@ -595,7 +595,7 @@
 <tr>
 <td>carbon.enable.page.level.reader.in.compaction</td>
 <td>true</td>
-<td>Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory.<strong>NOTE:</strong> Please refer to <a href="./file-structure-of-carbondata.html">file-structure-of-carbondata</a> to understand the storage format of CarbonData and concepts of pages.</td>
+<td>Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory.<strong>NOTE:</strong> Please refer to <a href="./file-structure-of-carbondata.html#carbondata-file-format">file-structure-of-carbondata</a> to understand the storage format of CarbonData and concepts of pages.</td>
 </tr>
 <tr>
 <td>carbon.concurrent.compaction</td>
@@ -916,8 +916,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -932,4 +931,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file


[34/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
Handled comments


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

Branch: refs/heads/asf-site
Commit: a51dc5960616b1e64c019212b7d838d9e2c77be9
Parents: 44eed09
Author: Raghunandan S <ca...@gmail.com>
Authored: Fri Sep 7 00:22:19 2018 +0800
Committer: Raghunandan S <ca...@gmail.com>
Committed: Sat Sep 8 00:27:00 2018 +0800

----------------------------------------------------------------------
 carbonscript.sh                                 |    5 +-
 .../classes/CleanUp$$anonfun$cleanUp$1.class    |  Bin 0 -> 1048 bytes
 .../classes/MDFileConverter$$anonfun$1.class    |  Bin 0 -> 1606 bytes
 .../classes/MDFileConverter$$anonfun$2.class    |  Bin 0 -> 1617 bytes
 content/WEB-INF/classes/MDFileConverter.class   |  Bin 10321 -> 10608 bytes
 ...ler$$anonfun$convertReadMeExtensions$1.class |  Bin 0 -> 2104 bytes
 content/WEB-INF/classes/MdFileHandler.class     |  Bin 6144 -> 6285 bytes
 ...eLauncher$$anonfun$deleteRecursively$1.class |  Bin 0 -> 1133 bytes
 content/WEB-INF/classes/application.conf        |   25 +-
 content/WEB-INF/classes/html/footer.html        |    2 +-
 content/WEB-INF/classes/html/header.html        |   57 +-
 content/bloomfilter-datamap-guide.html          |   13 +-
 content/configuration-parameters.html           |   19 +-
 content/data-management-on-carbondata.html      | 1321 ------------------
 content/data-management.html                    |  413 ------
 content/datamap-developer-guide.html            |   11 +-
 content/datamap-management.html                 |   11 +-
 content/ddl-of-carbondata.html                  |  185 ++-
 content/ddl-operation-on-carbondata.html        |  748 ----------
 content/dml-of-carbondata.html                  |  107 +-
 content/dml-operation-on-carbondata.html        |  716 ----------
 content/documentation.html                      |   18 +-
 content/errorpage.html                          |    2 +-
 content/faq.html                                |   29 +-
 content/file-structure-of-carbondata.html       |  155 +-
 .../how-to-contribute-to-apache-carbondata.html |   11 +-
 content/index.html                              |    2 +-
 content/installation-guide.html                 |  455 ------
 content/introduction.html                       |  188 +--
 content/language-manual.html                    |   12 +-
 content/lucene-datamap-guide.html               |   15 +-
 content/mainpage.html                           |  214 ---
 content/partition-guide.html                    |    2 +-
 content/performance-tuning.html                 |   84 +-
 content/preaggregate-datamap-guide.html         |   15 +-
 content/quick-start-guide.html                  |   53 +-
 content/release-guide.html                      |   11 +-
 content/s3-guide.html                           |   11 +-
 content/sdk-guide.html                          |   18 +-
 content/sdk-writer-guide.html                   |  549 --------
 content/security.html                           |    2 +-
 content/segment-management-on-carbondata.html   |   11 +-
 content/streaming-guide.html                    |  185 ++-
 content/supported-data-types-in-carbondata.html |   11 +-
 content/timeseries-datamap-guide.html           |   17 +-
 content/troubleshooting.html                    |  423 ------
 content/usecases.html                           |  619 ++++++++
 content/useful-tips-on-carbondata.html          |  542 -------
 content/videogallery.html                       |    2 +-
 src/main/resources/application.conf             |   21 +-
 src/main/scala/MDFileConverter.scala            |    6 +-
 src/main/scala/MdFileHandler.scala              |    6 +-
 src/main/scala/html/header.html                 |    6 +-
 .../scala/scripts/bloomfilter-datamap-guide     |   11 +
 src/main/scala/scripts/configuration-parameters |   11 +
 src/main/scala/scripts/datamap-developer-guide  |   11 +
 src/main/scala/scripts/datamap-management       |   11 +
 src/main/scala/scripts/ddl-of-carbondata        |   11 +
 src/main/scala/scripts/dml-of-carbondata        |   11 +
 src/main/scala/scripts/faq                      |    4 +
 .../scala/scripts/file-structure-of-carbondata  |   11 +
 .../how-to-contribute-to-apache-carbondata      |    4 +
 src/main/scala/scripts/introduction             |    4 +
 src/main/scala/scripts/language-manual          |   11 +
 src/main/scala/scripts/lucene-datamap-guide     |   11 +
 src/main/scala/scripts/performance-tuning       |    4 +
 .../scala/scripts/preaggregate-datamap-guide    |   11 +
 src/main/scala/scripts/quick-start-guide        |    4 +
 src/main/scala/scripts/release-guide            |    4 +
 src/main/scala/scripts/s3-guide                 |    4 +
 src/main/scala/scripts/sdk-guide                |    4 +
 .../scripts/segment-management-on-carbondata    |   10 +
 src/main/scala/scripts/streaming-guide          |   11 +
 .../scripts/supported-data-types-in-carbondata  |   11 +
 src/main/scala/scripts/timeseries-datamap-guide |   11 +
 src/main/scala/scripts/usecases                 |    4 +
 src/main/webapp/bloomfilter-datamap-guide.html  |   13 +-
 src/main/webapp/configuration-parameters.html   |   19 +-
 .../webapp/data-management-on-carbondata.html   | 1321 ------------------
 src/main/webapp/datamap-developer-guide.html    |   11 +-
 src/main/webapp/datamap-management.html         |   11 +-
 src/main/webapp/ddl-of-carbondata.html          |  185 ++-
 src/main/webapp/dml-of-carbondata.html          |  107 +-
 src/main/webapp/documentation.html              |   18 +-
 src/main/webapp/errorpage.html                  |    2 +-
 src/main/webapp/faq.html                        |   29 +-
 .../webapp/file-structure-of-carbondata.html    |  155 +-
 .../how-to-contribute-to-apache-carbondata.html |   11 +-
 src/main/webapp/index.html                      |    2 +-
 src/main/webapp/installation-guide.html         |  455 ------
 src/main/webapp/introduction.html               |  188 +--
 src/main/webapp/language-manual.html            |   12 +-
 src/main/webapp/lucene-datamap-guide.html       |   15 +-
 src/main/webapp/mainpage.html                   |  214 ---
 src/main/webapp/performance-tuning.html         |   84 +-
 src/main/webapp/preaggregate-datamap-guide.html |   15 +-
 src/main/webapp/quick-start-guide.html          |   53 +-
 src/main/webapp/release-guide.html              |   11 +-
 src/main/webapp/s3-guide.html                   |   11 +-
 src/main/webapp/sdk-guide.html                  |   18 +-
 src/main/webapp/sdk-writer-guide.html           |  549 --------
 src/main/webapp/security.html                   |    2 +-
 .../segment-management-on-carbondata.html       |   11 +-
 src/main/webapp/streaming-guide.html            |  185 ++-
 .../supported-data-types-in-carbondata.html     |   11 +-
 src/main/webapp/timeseries-datamap-guide.html   |   17 +-
 src/main/webapp/troubleshooting.html            |  423 ------
 src/main/webapp/usecases.html                   |  619 ++++++++
 src/main/webapp/useful-tips-on-carbondata.html  |  542 -------
 src/main/webapp/videogallery.html               |    2 +-
 src/site/markdown/bloomfilter-datamap-guide.md  |   13 +-
 src/site/markdown/configuration-parameters.md   |   21 +-
 src/site/markdown/datamap-developer-guide.md    |   13 +-
 src/site/markdown/datamap-management.md         |   12 -
 src/site/markdown/ddl-of-carbondata.md          |  119 +-
 src/site/markdown/dml-of-carbondata.md          |   37 +-
 src/site/markdown/documentation.md              |   10 +-
 src/site/markdown/faq.md                        |   25 +-
 .../markdown/file-structure-of-carbondata.md    |  185 ++-
 .../how-to-contribute-to-apache-carbondata.md   |    9 +-
 src/site/markdown/introduction.md               |  147 +-
 src/site/markdown/language-manual.md            |   22 +-
 src/site/markdown/lucene-datamap-guide.md       |   15 +-
 src/site/markdown/performance-tuning.md         |   81 +-
 src/site/markdown/preaggregate-datamap-guide.md |   16 +-
 src/site/markdown/quick-start-guide.md          |   59 +-
 src/site/markdown/release-guide.md              |    6 -
 src/site/markdown/s3-guide.md                   |    4 -
 src/site/markdown/sdk-guide.md                  |   17 +-
 .../segment-management-on-carbondata.md         |   12 -
 src/site/markdown/streaming-guide.md            |  179 ++-
 .../supported-data-types-in-carbondata.md       |   13 +-
 src/site/markdown/timeseries-datamap-guide.md   |   17 +-
 src/site/markdown/usecases.md                   |  215 +++
 134 files changed, 4074 insertions(+), 10011 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/carbonscript.sh
----------------------------------------------------------------------
diff --git a/carbonscript.sh b/carbonscript.sh
index df9de3e..b77e42d 100755
--- a/carbonscript.sh
+++ b/carbonscript.sh
@@ -6,7 +6,4 @@ mvn exec:java -Dexec.mainClass=WebsiteLauncher
 
 mvn install
 
-# mvn exec:java -Dexec.mainClass=CleanUp
-
-find . -type f -name "*.html" ! -name "header.html" ! -name "footer.html" -exec sed -i '' 's/&lt;script&gt;/<script>/g' {} +
-find . -type f -name "*.html" ! -name "header.html" ! -name "footer.html" -exec sed -i '' 's/&lt;\/script&gt;/<\/script>/g' {} +
+# mvn exec:java -Dexec.mainClass=CleanUp
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/WEB-INF/classes/CleanUp$$anonfun$cleanUp$1.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/CleanUp$$anonfun$cleanUp$1.class b/content/WEB-INF/classes/CleanUp$$anonfun$cleanUp$1.class
new file mode 100644
index 0000000..5b6d7e1
Binary files /dev/null and b/content/WEB-INF/classes/CleanUp$$anonfun$cleanUp$1.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/WEB-INF/classes/MDFileConverter$$anonfun$1.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/MDFileConverter$$anonfun$1.class b/content/WEB-INF/classes/MDFileConverter$$anonfun$1.class
new file mode 100644
index 0000000..a1c473f
Binary files /dev/null and b/content/WEB-INF/classes/MDFileConverter$$anonfun$1.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/WEB-INF/classes/MDFileConverter$$anonfun$2.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/MDFileConverter$$anonfun$2.class b/content/WEB-INF/classes/MDFileConverter$$anonfun$2.class
new file mode 100644
index 0000000..157c615
Binary files /dev/null and b/content/WEB-INF/classes/MDFileConverter$$anonfun$2.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/WEB-INF/classes/MDFileConverter.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/MDFileConverter.class b/content/WEB-INF/classes/MDFileConverter.class
index 76a6063..7cce73f 100644
Binary files a/content/WEB-INF/classes/MDFileConverter.class and b/content/WEB-INF/classes/MDFileConverter.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/WEB-INF/classes/MdFileHandler$$anonfun$convertReadMeExtensions$1.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/MdFileHandler$$anonfun$convertReadMeExtensions$1.class b/content/WEB-INF/classes/MdFileHandler$$anonfun$convertReadMeExtensions$1.class
new file mode 100644
index 0000000..bdd1312
Binary files /dev/null and b/content/WEB-INF/classes/MdFileHandler$$anonfun$convertReadMeExtensions$1.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/WEB-INF/classes/MdFileHandler.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/MdFileHandler.class b/content/WEB-INF/classes/MdFileHandler.class
index f39c098..d0cd5a8 100644
Binary files a/content/WEB-INF/classes/MdFileHandler.class and b/content/WEB-INF/classes/MdFileHandler.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/WEB-INF/classes/WebsiteLauncher$$anonfun$deleteRecursively$1.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/WebsiteLauncher$$anonfun$deleteRecursively$1.class b/content/WEB-INF/classes/WebsiteLauncher$$anonfun$deleteRecursively$1.class
new file mode 100644
index 0000000..b7f7385
Binary files /dev/null and b/content/WEB-INF/classes/WebsiteLauncher$$anonfun$deleteRecursively$1.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/WEB-INF/classes/application.conf
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/application.conf b/content/WEB-INF/classes/application.conf
index b5f941f..ba425e5 100644
--- a/content/WEB-INF/classes/application.conf
+++ b/content/WEB-INF/classes/application.conf
@@ -15,7 +15,8 @@ fileList=["configuration-parameters",
   "segment-management-on-carbondata",
   "release-guide",
   "how-to-contribute-to-apache-carbondata",
-  "introduction"
+  "introduction",
+  "usecases"
   ]
 dataMapFileList=[
   "bloomfilter-datamap-guide",
@@ -24,14 +25,30 @@ dataMapFileList=[
   "timeseries-datamap-guide",
   "datamap-management"]
 
-dataMapFilesUrl="file:///Users/aditi_advith/Documents/code/carbondata/docs/datamap/"
+dataMapFilesUrl="https://raw.githubusercontent.com/apache/carbondata/master/docs/datamap/"
 
 fileListToRetain=["quick-start-guide"
 ]
 headerPath="src/main/scala/html/header.html"
 footerPath="src/main/scala/html/footer.html"
 outputFileLocation="src/main/webapp/"
-apiUrl="file:///Users/aditi_advith/Documents/code/carbondata/docs/"
+apiUrl="https://raw.githubusercontent.com/apache/carbondata/master/docs/"
 mdLink="https://api.github.com/markdown/raw"
 mdFileLocation="src/site/markdown/"
-imagesFilesList=["file-structure-of-carbondata"]
\ No newline at end of file
+imagesFilesList=["file-structure-of-carbondata",
+"2-1_1",
+"2-2_1",
+"2-3_1",
+"2-3_2",
+"2-3_3",
+"2-3_4",
+"2-4_1",
+"2-5_1",
+"2-5_2",
+"2-5_3",
+"2-6_1",
+"carbon_data_file_structure_new",
+"carbon_data_format_new",
+"carbondata-performance"
+]
+scriptsPath="src/main/scala/scripts/"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/WEB-INF/classes/html/footer.html
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/html/footer.html b/content/WEB-INF/classes/html/footer.html
index eb80d58..47aa819 100644
--- a/content/WEB-INF/classes/html/footer.html
+++ b/content/WEB-INF/classes/html/footer.html
@@ -13,4 +13,4 @@
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/WEB-INF/classes/html/header.html
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/html/header.html b/content/WEB-INF/classes/html/header.html
index 5d5a2e0..217a9f4 100644
--- a/content/WEB-INF/classes/html/header.html
+++ b/content/WEB-INF/classes/html/header.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,10 +153,60 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
                             <div class="row">
                                 <div class="col-sm-12  col-md-12">
-                                    <div>
+                                    <div>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/bloomfilter-datamap-guide.html
----------------------------------------------------------------------
diff --git a/content/bloomfilter-datamap-guide.html b/content/bloomfilter-datamap-guide.html
index 54c14fe..8030599 100644
--- a/content/bloomfilter-datamap-guide.html
+++ b/content/bloomfilter-datamap-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -260,7 +260,7 @@ User can create BloomFilter datamap on specified columns with specified BloomFil
   age int,
   city string,
   country string)
-STORED BY 'carbondata'
+STORED AS carbondata
 TBLPROPERTIES('SORT_COLUMNS'='id')
 </code></pre>
 <p>In the above example, <code>id</code> and <code>name</code> are high cardinality columns
@@ -366,8 +366,7 @@ $(function() {
     $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -382,4 +381,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/content/configuration-parameters.html
----------------------------------------------------------------------
diff --git a/content/configuration-parameters.html b/content/configuration-parameters.html
index ba73b0d..ab89576 100644
--- a/content/configuration-parameters.html
+++ b/content/configuration-parameters.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -212,7 +212,7 @@
                                     <div>
 <h1>
 <a id="configuring-carbondata" class="anchor" href="#configuring-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configuring CarbonData</h1>
-<p>This guide explains the configurations that can be used to tune CarbonData to achieve better performance.Some of the properties can be set dynamically and are explained in the section Dynamic Configuration In CarbonData Using SET-RESET.Most of the properties that control the internal settings have reasonable default values.They are listed along with the properties along with explanation.</p>
+<p>This guide explains the configurations that can be used to tune CarbonData to achieve better performance.Most of the properties that control the internal settings have reasonable default values.They are listed along with the properties along with explanation.</p>
 <ul>
 <li><a href="#system-configuration">System Configuration</a></li>
 <li><a href="#data-loading-configuration">Data Loading Configuration</a></li>
@@ -364,7 +364,7 @@
 <tr>
 <td>carbon.blockletgroup.size.in.mb</td>
 <td>64</td>
-<td>Please refer to <a href="./file-structure-of-carbondata.html">file-structure-of-carbondata</a> to understand the storage format of CarbonData.The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of each blocklet group. Higher value results in better sequential IO access.The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB).<strong>NOTE:</strong> Configuring a higher value might lead to poor performance as an entire blocklet group will have to read into memory before processing.For filter queries with limit, it is <strong>not advisable</strong> to have a bigger blocklet size.For Aggregation queries which need to return more number of rows,bigger blocklet size is advisable.</td>
+<td>Please refer to <a href="./file-structure-of-carbondata.html#carbondata-file-format">file-structure-of-carbondata</a> to understand the storage format of CarbonData.The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of each blocklet group. Higher value results in better sequential IO access.The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB).<strong>NOTE:</strong> Configuring a higher value might lead to poor performance as an entire blocklet group will have to read into memory before processing.For filter queries with limit, it is <strong>not advisable</strong> to have a bigger blocklet size.For Aggregation queries which need to return more number of rows,bigger blocklet size is advisable.</td>
 </tr>
 <tr>
 <td>carbon.sort.file.write.buffer.size</td>
@@ -420,7 +420,7 @@
 <tr>
 <td>carbon.use.local.dir</td>
 <td>false</td>
-<td>CarbonData during data loading, writes files to local temp directories before copying the files to HDFS.This configuration is used to specify whether CarbonData can write locally to tmp directory of the container or to the YARN application directory.</td>
+<td>CarbonData,during data loading, writes files to local temp directories before copying the files to HDFS.This configuration is used to specify whether CarbonData can write locally to tmp directory of the container or to the YARN application directory.</td>
 </tr>
 <tr>
 <td>carbon.use.multiple.temp.dir</td>
@@ -595,7 +595,7 @@
 <tr>
 <td>carbon.enable.page.level.reader.in.compaction</td>
 <td>true</td>
-<td>Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory.<strong>NOTE:</strong> Please refer to <a href="./file-structure-of-carbondata.html">file-structure-of-carbondata</a> to understand the storage format of CarbonData and concepts of pages.</td>
+<td>Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory.<strong>NOTE:</strong> Please refer to <a href="./file-structure-of-carbondata.html#carbondata-file-format">file-structure-of-carbondata</a> to understand the storage format of CarbonData and concepts of pages.</td>
 </tr>
 <tr>
 <td>carbon.concurrent.compaction</td>
@@ -916,8 +916,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -932,4 +931,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file


[37/39] carbondata-site git commit: refactor document

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/bee56334/content/installation-guide.html
----------------------------------------------------------------------
diff --git a/content/installation-guide.html b/content/installation-guide.html
new file mode 100644
index 0000000..2e7fab6
--- /dev/null
+++ b/content/installation-guide.html
@@ -0,0 +1,455 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="row">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="installation-guide" class="anchor" href="#installation-guide" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installation Guide</h1>
+<p>This tutorial guides you through the installation and configuration of CarbonData in the following two modes :</p>
+<ul>
+<li><a href="#installing-and-configuring-carbondata-on-standalone-spark-cluster">Installing and Configuring CarbonData on Standalone Spark Cluster</a></li>
+<li><a href="#installing-and-configuring-carbondata-on-spark-on-yarn-cluster">Installing and Configuring CarbonData on Spark on YARN Cluster</a></li>
+</ul>
+<p>followed by :</p>
+<ul>
+<li><a href="#query-execution-using-carbondata-thrift-server">Query Execution using CarbonData Thrift Server</a></li>
+</ul>
+<h2>
+<a id="installing-and-configuring-carbondata-on-standalone-spark-cluster" class="anchor" href="#installing-and-configuring-carbondata-on-standalone-spark-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Standalone Spark Cluster</h2>
+<h3>
+<a id="prerequisites" class="anchor" href="#prerequisites" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h3>
+<ul>
+<li>
+<p>Hadoop HDFS and Yarn should be installed and running.</p>
+</li>
+<li>
+<p>Spark should be installed and running on all the cluster nodes.</p>
+</li>
+<li>
+<p>CarbonData user should have permission to access HDFS.</p>
+</li>
+</ul>
+<h3>
+<a id="procedure" class="anchor" href="#procedure" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Procedure</h3>
+<ol>
+<li>
+<p><a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>Build the CarbonData</a> project and get the assembly jar from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code>.</p>
+</li>
+<li>
+<p>Copy <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code> to <code>$SPARK_HOME/carbonlib</code> folder.</p>
+<p><strong>NOTE</strong>: Create the carbonlib folder if it does not exist inside <code>$SPARK_HOME</code> path.</p>
+</li>
+<li>
+<p>Add the carbonlib folder path in the Spark classpath. (Edit <code>$SPARK_HOME/conf/spark-env.sh</code> file and modify the value of <code>SPARK_CLASSPATH</code> by appending <code>$SPARK_HOME/carbonlib/*</code> to the existing value)</p>
+</li>
+<li>
+<p>Copy the <code>./conf/carbon.properties.template</code> file from CarbonData repository to <code>$SPARK_HOME/conf/</code> folder and rename the file to <code>carbon.properties</code>.</p>
+</li>
+<li>
+<p>Repeat Step 2 to Step 5 in all the nodes of the cluster.</p>
+</li>
+<li>
+<p>In Spark node[master], configure the properties mentioned in the following table in <code>$SPARK_HOME/conf/spark-defaults.conf</code> file.</p>
+</li>
+</ol>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Value</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>spark.driver.extraJavaOptions</td>
+<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
+<td>A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.</td>
+</tr>
+<tr>
+<td>spark.executor.extraJavaOptions</td>
+<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
+<td>A string of extra JVM options to pass to executors. For instance, GC settings or other logging. <strong>NOTE</strong>: You can enter multiple values separated by space.</td>
+</tr>
+</tbody>
+</table>
+<ol start="7">
+<li>Add the following properties in <code>$SPARK_HOME/conf/carbon.properties</code> file:</li>
+</ol>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Required</th>
+<th>Description</th>
+<th>Example</th>
+<th>Remark</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>carbon.storelocation</td>
+<td>NO</td>
+<td>Location where data CarbonData will create the store and write the data in its own format. If not specified then it takes spark.sql.warehouse.dir path.</td>
+<td>hdfs://HOSTNAME:PORT/Opt/CarbonStore</td>
+<td>Propose to set HDFS directory</td>
+</tr>
+</tbody>
+</table>
+<ol start="8">
+<li>Verify the installation. For example:</li>
+</ol>
+<pre><code>./spark-shell --master spark://HOSTNAME:PORT --total-executor-cores 2
+--executor-memory 2G
+</code></pre>
+<p><strong>NOTE</strong>: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.</p>
+<p>To get started with CarbonData : <a href="quick-start-guide.html">Quick Start</a>, <a href="data-management-on-carbondata.html">Data Management on CarbonData</a></p>
+<h2>
+<a id="installing-and-configuring-carbondata-on-spark-on-yarn-cluster" class="anchor" href="#installing-and-configuring-carbondata-on-spark-on-yarn-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Spark on YARN Cluster</h2>
+<p>This section provides the procedure to install CarbonData on "Spark on YARN" cluster.</p>
+<h3>
+<a id="prerequisites-1" class="anchor" href="#prerequisites-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h3>
+<ul>
+<li>Hadoop HDFS and Yarn should be installed and running.</li>
+<li>Spark should be installed and running in all the clients.</li>
+<li>CarbonData user should have permission to access HDFS.</li>
+</ul>
+<h3>
+<a id="procedure-1" class="anchor" href="#procedure-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Procedure</h3>
+<p>The following steps are only for Driver Nodes. (Driver nodes are the one which starts the spark context.)</p>
+<ol>
+<li>
+<p><a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>Build the CarbonData</a> project and get the assembly jar from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code> and copy to <code>$SPARK_HOME/carbonlib</code> folder.</p>
+<p><strong>NOTE</strong>: Create the carbonlib folder if it does not exists inside <code>$SPARK_HOME</code> path.</p>
+</li>
+<li>
+<p>Copy the <code>./conf/carbon.properties.template</code> file from CarbonData repository to <code>$SPARK_HOME/conf/</code> folder and rename the file to <code>carbon.properties</code>.</p>
+</li>
+<li>
+<p>Create <code>tar.gz</code> file of carbonlib folder and move it inside the carbonlib folder.</p>
+</li>
+</ol>
+<pre><code>cd $SPARK_HOME
+tar -zcvf carbondata.tar.gz carbonlib/
+mv carbondata.tar.gz carbonlib/
+</code></pre>
+<ol start="4">
+<li>Configure the properties mentioned in the following table in <code>$SPARK_HOME/conf/spark-defaults.conf</code> file.</li>
+</ol>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Description</th>
+<th>Value</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>spark.master</td>
+<td>Set this value to run the Spark in yarn cluster mode.</td>
+<td>Set yarn-client to run the Spark in yarn cluster mode.</td>
+</tr>
+<tr>
+<td>spark.yarn.dist.files</td>
+<td>Comma-separated list of files to be placed in the working directory of each executor.</td>
+<td><code>$SPARK_HOME/conf/carbon.properties</code></td>
+</tr>
+<tr>
+<td>spark.yarn.dist.archives</td>
+<td>Comma-separated list of archives to be extracted into the working directory of each executor.</td>
+<td><code>$SPARK_HOME/carbonlib/carbondata.tar.gz</code></td>
+</tr>
+<tr>
+<td>spark.executor.extraJavaOptions</td>
+<td>A string of extra JVM options to pass to executors. For instance  <strong>NOTE</strong>: You can enter multiple values separated by space.</td>
+<td><code>-Dcarbon.properties.filepath = carbon.properties</code></td>
+</tr>
+<tr>
+<td>spark.executor.extraClassPath</td>
+<td>Extra classpath entries to prepend to the classpath of executors. <strong>NOTE</strong>: If SPARK_CLASSPATH is defined in spark-env.sh, then comment it and append the values in below parameter spark.driver.extraClassPath</td>
+<td><code>carbondata.tar.gz/carbonlib/*</code></td>
+</tr>
+<tr>
+<td>spark.driver.extraClassPath</td>
+<td>Extra classpath entries to prepend to the classpath of the driver. <strong>NOTE</strong>: If SPARK_CLASSPATH is defined in spark-env.sh, then comment it and append the value in below parameter spark.driver.extraClassPath.</td>
+<td><code>$SPARK_HOME/carbonlib/*</code></td>
+</tr>
+<tr>
+<td>spark.driver.extraJavaOptions</td>
+<td>A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.</td>
+<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
+</tr>
+</tbody>
+</table>
+<ol start="5">
+<li>Add the following properties in <code>$SPARK_HOME/conf/carbon.properties</code>:</li>
+</ol>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Required</th>
+<th>Description</th>
+<th>Example</th>
+<th>Default Value</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>carbon.storelocation</td>
+<td>NO</td>
+<td>Location where CarbonData will create the store and write the data in its own format. If not specified then it takes spark.sql.warehouse.dir path.</td>
+<td>hdfs://HOSTNAME:PORT/Opt/CarbonStore</td>
+<td>Propose to set HDFS directory</td>
+</tr>
+</tbody>
+</table>
+<ol start="6">
+<li>Verify the installation.</li>
+</ol>
+<pre><code> ./bin/spark-shell --master yarn-client --driver-memory 1g
+ --executor-cores 2 --executor-memory 2G
+</code></pre>
+<p><strong>NOTE</strong>: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.</p>
+<p>Getting started with CarbonData : <a href="quick-start-guide.html">Quick Start</a>, <a href="data-management-on-carbondata.html">Data Management on CarbonData</a></p>
+<h2>
+<a id="query-execution-using-carbondata-thrift-server" class="anchor" href="#query-execution-using-carbondata-thrift-server" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Query Execution Using CarbonData Thrift Server</h2>
+<h3>
+<a id="starting-carbondata-thrift-server" class="anchor" href="#starting-carbondata-thrift-server" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Starting CarbonData Thrift Server.</h3>
+<p>a. cd <code>$SPARK_HOME</code></p>
+<p>b. Run the following command to start the CarbonData thrift server.</p>
+<pre><code>./bin/spark-submit
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer
+$SPARK_HOME/carbonlib/$CARBON_ASSEMBLY_JAR &lt;carbon_store_path&gt;
+</code></pre>
+<table>
+<thead>
+<tr>
+<th>Parameter</th>
+<th>Description</th>
+<th>Example</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>CARBON_ASSEMBLY_JAR</td>
+<td>CarbonData assembly jar name present in the <code>$SPARK_HOME/carbonlib/</code> folder.</td>
+<td>carbondata_2.xx-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar</td>
+</tr>
+<tr>
+<td>carbon_store_path</td>
+<td>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. If not specified then it takes spark.sql.warehouse.dir path.</td>
+<td><code>hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store</code></td>
+</tr>
+</tbody>
+</table>
+<p><strong>NOTE</strong>: From Spark 1.6, by default the Thrift server runs in multi-session mode. Which means each JDBC/ODBC connection owns a copy of their own SQL configuration and temporary function registry. Cached tables are still shared though. If you prefer to run the Thrift server in single-session mode and share all SQL configuration and temporary function registry, please set option <code>spark.sql.hive.thriftServer.singleSession</code> to <code>true</code>. You may either add this option to <code>spark-defaults.conf</code>, or pass it to <code>spark-submit.sh</code> via <code>--conf</code>:</p>
+<pre><code>./bin/spark-submit
+--conf spark.sql.hive.thriftServer.singleSession=true
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer
+$SPARK_HOME/carbonlib/$CARBON_ASSEMBLY_JAR &lt;carbon_store_path&gt;
+</code></pre>
+<p><strong>But</strong> in single-session mode, if one user changes the database from one connection, the database of the other connections will be changed too.</p>
+<p><strong>Examples</strong></p>
+<ul>
+<li>Start with default memory and executors.</li>
+</ul>
+<pre><code>./bin/spark-submit
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer 
+$SPARK_HOME/carbonlib
+/carbondata_2.xx-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar
+hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
+</code></pre>
+<ul>
+<li>Start with Fixed executors and resources.</li>
+</ul>
+<pre><code>./bin/spark-submit
+--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://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
+</code></pre>
+<h3>
+<a id="connecting-to-carbondata-thrift-server-using-beeline" class="anchor" href="#connecting-to-carbondata-thrift-server-using-beeline" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Connecting to CarbonData Thrift Server Using Beeline.</h3>
+<pre><code>     cd $SPARK_HOME
+     ./sbin/start-thriftserver.sh
+     ./bin/beeline -u jdbc:hive2://&lt;thriftserver_host&gt;:port
+
+     Example
+     ./bin/beeline -u jdbc:hive2://10.10.10.10:10000
+</code></pre>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/bee56334/content/mainpage.html
----------------------------------------------------------------------
diff --git a/content/mainpage.html b/content/mainpage.html
new file mode 100644
index 0000000..d515853
--- /dev/null
+++ b/content/mainpage.html
@@ -0,0 +1,214 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input" placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="row">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="doc-heading">
+                                <h4 class="title">Documentation
+                                    <span class="title-underline"></span>
+                                </h4>
+                            </div>
+
+                            <div class="row">
+
+                                <div class="col-sm-12  col-md-12">
+                                    <span class="text-justify">
+                                        Welcome to Apache CarbonData. Apache CarbonData is a new big data file format for faster interactive query using advanced columnar storage, index, compression and encoding techniques to improve computing efficiency, which helps in speeding up queries by an order of magnitude faster over PetaBytes of data. This user guide provides a detailed description about the CarbonData and its features.
+                                        Let's get started !
+                                    </span>
+                                    <hr style="margin: 12px 0 8px">
+                                    <div>
+                                        <ul class="sub-nav">
+                                            <li><a href="quick-start-guide.html">Quick Start</a></li>
+                                            <li><a href="file-structure-of-carbondata.html">CarbonData File Structure</a></li>
+                                            <li><a href="supported-data-types-in-carbondata.html">Data Types</a></li>
+                                            <li><a href="data-management-on-carbondata.html">Data Management On CarbonData</a></li>
+                                            <li><a href="installation-guide.html">Installation Guide</a></li>
+                                            <li><a href="configuration-parameters.html">Configuring CarbonData</a></li>
+                                            <li><a href="streaming-guide.html">Streaming Guide</a></li>
+                                            <li><a href="sdk-guide.html">SDK Guide</a></li>
+											<li><a href="s3-guide.html">S3 Guide (Alpha Feature)</a></li>
+                                            <li><a href="datamap-developer-guide.html">DataMap Developer Guide</a></li>
+											<li><a href="datamap-management.html">CarbonData DataMap Management</a></li>
+                                            <li><a href="bloomfilter-datamap-guide.html">CarbonData BloomFilter DataMap (Alpha Feature)</a></li>
+                                            <li><a href="lucene-datamap-guide.html">CarbonData Lucene DataMap (Alpha Feature)</a></li>
+                                            <li><a href="preaggregate-datamap-guide.html">CarbonData Pre-aggregate DataMap</a></li>
+                                            <li><a href="timeseries-datamap-guide.html">CarbonData Timeseries DataMap</a></li>
+                                            <li><a href="faq.html">FAQs</a></li>
+                                            <li><a href="troubleshooting.html">Troubleshooting</a></li>
+                                            <li><a href="useful-tips-on-carbondata.html">Useful Tips</a></li>
+
+                                        </ul>
+                                    </div>
+                                </div>
+                            </div>
+                        </div>
+                        <div class="doc-footer">
+                            <a href="#top" class="scroll-top">Top</a>
+                        </div>
+                    </div>
+                </section>
+            </div>
+        </div>
+    </div>
+</section><!-- End systemblock part -->
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/bee56334/content/sdk-writer-guide.html
----------------------------------------------------------------------
diff --git a/content/sdk-writer-guide.html b/content/sdk-writer-guide.html
new file mode 100644
index 0000000..36bb9ad
--- /dev/null
+++ b/content/sdk-writer-guide.html
@@ -0,0 +1,549 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
+                                   target="_blank">Apache CarbonData 1.2.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
+                                   target="_blank">Apache CarbonData 1.1.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
+                                   target="_blank">Apache CarbonData 1.1.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
+                                   target="_blank">Apache CarbonData 1.0.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.2.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.1</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="row">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div><h1>
+<a id="sdk-writer-guide" class="anchor" href="#sdk-writer-guide" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SDK Writer Guide</h1>
+<p>In the carbon jars package, there exist a carbondata-store-sdk-x.x.x-SNAPSHOT.jar.
+This SDK writer, writes carbondata file and carbonindex file at a given path.
+External client can make use of this writer to convert other format data or live data to create carbondata and index files.
+These SDK writer output contains just a carbondata and carbonindex files. No metadata folder will be present.</p>
+<h2>
+<a id="quick-example" class="anchor" href="#quick-example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Quick example</h2>
+<h3>
+<a id="example-with-csv-format" class="anchor" href="#example-with-csv-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example with csv format</h3>
+<div class="highlight highlight-source-java"><pre> <span class="pl-k">import</span> <span class="pl-smi">java.io.IOException</span>;
+ 
+ <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException</span>;
+ <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.core.metadata.datatype.DataTypes</span>;
+ <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.CarbonWriter</span>;
+ <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.CarbonWriterBuilder</span>;
+ <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.Field</span>;
+ <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.Schema</span>;
+ 
+ <span class="pl-k">public</span> <span class="pl-k">class</span> <span class="pl-en">TestSdk</span> {
+ 
+   <span class="pl-k">public</span> <span class="pl-k">static</span> <span class="pl-k">void</span> <span class="pl-en">main</span>(<span class="pl-k">String</span>[] <span class="pl-v">args</span>) <span class="pl-k">throws</span> <span class="pl-smi">IOException</span>, <span class="pl-smi">InvalidLoadOptionException</span> {
+     testSdkWriter();
+   }
+ 
+   <span class="pl-k">public</span> <span class="pl-k">static</span> <span class="pl-k">void</span> <span class="pl-en">testSdkWriter</span>() <span class="pl-k">throws</span> <span class="pl-smi">IOException</span>, <span class="pl-smi">InvalidLoadOptionException</span> {
+     <span class="pl-smi">String</span> path <span class="pl-k">=</span> <span class="pl-s"><span class="pl-pds">"</span>/home/root1/Documents/ab/temp<span class="pl-pds">"</span></span>;
+ 
+     <span class="pl-k">Field</span>[] fields <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">Field</span>[<span class="pl-c1">2</span>];
+     fields[<span class="pl-c1">0</span>] <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">Field</span>(<span class="pl-s"><span class="pl-pds">"</span>name<span class="pl-pds">"</span></span>, <span class="pl-smi">DataTypes</span><span class="pl-c1"><span class="pl-k">.</span>STRING</span>);
+     fields[<span class="pl-c1">1</span>] <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">Field</span>(<span class="pl-s"><span class="pl-pds">"</span>age<span class="pl-pds">"</span></span>, <span class="pl-smi">DataTypes</span><span class="pl-c1"><span class="pl-k">.</span>INT</span>);
+ 
+     <span class="pl-smi">Schema</span> schema <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">Schema</span>(fields);
+ 
+     <span class="pl-smi">CarbonWriterBuilder</span> builder <span class="pl-k">=</span> <span class="pl-smi">CarbonWriter</span><span class="pl-k">.</span>builder()<span class="pl-k">.</span>withSchema(schema)<span class="pl-k">.</span>outputPath(path);
+ 
+     <span class="pl-smi">CarbonWriter</span> writer <span class="pl-k">=</span> builder<span class="pl-k">.</span>buildWriterForCSVInput();
+ 
+     <span class="pl-k">int</span> rows <span class="pl-k">=</span> <span class="pl-c1">5</span>;
+     <span class="pl-k">for</span> (<span class="pl-k">int</span> i <span class="pl-k">=</span> <span class="pl-c1">0</span>; i <span class="pl-k">&lt;</span> rows; i<span class="pl-k">++</span>) {
+       writer<span class="pl-k">.</span>write(<span class="pl-k">new</span> <span class="pl-smi">String</span>[] { <span class="pl-s"><span class="pl-pds">"</span>robot<span class="pl-pds">"</span></span> <span class="pl-k">+</span> (i <span class="pl-k">%</span> <span class="pl-c1">10</span>), <span class="pl-smi">String</span><span class="pl-k">.</span>valueOf(i) });
+     }
+     writer<span class="pl-k">.</span>close();
+   }
+ }</pre></div>
+<h3>
+<a id="example-with-avro-format" class="anchor" href="#example-with-avro-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example with Avro format</h3>
+<div class="highlight highlight-source-java"><pre><span class="pl-k">import</span> <span class="pl-smi">java.io.IOException</span>;
+
+<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException</span>;
+<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.core.metadata.datatype.DataTypes</span>;
+<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.AvroCarbonWriter</span>;
+<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.CarbonWriter</span>;
+<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.Field</span>;
+
+<span class="pl-k">import</span> <span class="pl-smi">org.apache.avro.generic.GenericData</span>;
+<span class="pl-k">import</span> <span class="pl-smi">org.apache.commons.lang.CharEncoding</span>;
+
+<span class="pl-k">import</span> <span class="pl-smi">tech.allegro.schema.json2avro.converter.JsonAvroConverter</span>;
+
+<span class="pl-k">public</span> <span class="pl-k">class</span> <span class="pl-en">TestSdkAvro</span> {
+
+  <span class="pl-k">public</span> <span class="pl-k">static</span> <span class="pl-k">void</span> <span class="pl-en">main</span>(<span class="pl-k">String</span>[] <span class="pl-v">args</span>) <span class="pl-k">throws</span> <span class="pl-smi">IOException</span>, <span class="pl-smi">InvalidLoadOptionException</span> {
+    testSdkWriter();
+  }
+
+
+  <span class="pl-k">public</span> <span class="pl-k">static</span> <span class="pl-k">void</span> <span class="pl-en">testSdkWriter</span>() <span class="pl-k">throws</span> <span class="pl-smi">IOException</span>, <span class="pl-smi">InvalidLoadOptionException</span> {
+    <span class="pl-smi">String</span> path <span class="pl-k">=</span> <span class="pl-s"><span class="pl-pds">"</span>./AvroCarbonWriterSuiteWriteFiles<span class="pl-pds">"</span></span>;
+    <span class="pl-c"><span class="pl-c">//</span> Avro schema</span>
+    <span class="pl-smi">String</span> avroSchema <span class="pl-k">=</span>
+        <span class="pl-s"><span class="pl-pds">"</span>{<span class="pl-pds">"</span></span> <span class="pl-k">+</span>
+            <span class="pl-s"><span class="pl-pds">"</span>   <span class="pl-cce">\"</span>type<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>record<span class="pl-cce">\"</span>,<span class="pl-pds">"</span></span> <span class="pl-k">+</span>
+            <span class="pl-s"><span class="pl-pds">"</span>   <span class="pl-cce">\"</span>name<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>Acme<span class="pl-cce">\"</span>,<span class="pl-pds">"</span></span> <span class="pl-k">+</span>
+            <span class="pl-s"><span class="pl-pds">"</span>   <span class="pl-cce">\"</span>fields<span class="pl-cce">\"</span> : [<span class="pl-pds">"</span></span>
+            <span class="pl-k">+</span> <span class="pl-s"><span class="pl-pds">"</span>{ <span class="pl-cce">\"</span>name<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>fname<span class="pl-cce">\"</span>, <span class="pl-cce">\"</span>type<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>string<span class="pl-cce">\"</span> },<span class="pl-pds">"</span></span>
+            <span class="pl-k">+</span> <span class="pl-s"><span class="pl-pds">"</span>{ <span class="pl-cce">\"</span>name<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>age<span class="pl-cce">\"</span>, <span class="pl-cce">\"</span>type<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>int<span class="pl-cce">\"</span> }]<span class="pl-pds">"</span></span> <span class="pl-k">+</span>
+            <span class="pl-s"><span class="pl-pds">"</span>}<span class="pl-pds">"</span></span>;
+
+    <span class="pl-smi">String</span> json <span class="pl-k">=</span> <span class="pl-s"><span class="pl-pds">"</span>{<span class="pl-cce">\"</span>fname<span class="pl-cce">\"</span>:<span class="pl-cce">\"</span>bob<span class="pl-cce">\"</span>, <span class="pl-cce">\"</span>age<span class="pl-cce">\"</span>:10}<span class="pl-pds">"</span></span>;
+
+    <span class="pl-c"><span class="pl-c">//</span> conversion to GenericData.Record</span>
+    <span class="pl-smi">JsonAvroConverter</span> converter <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">JsonAvroConverter</span>();
+    <span class="pl-smi">GenericData</span><span class="pl-k">.</span><span class="pl-smi">Record</span> record <span class="pl-k">=</span> converter<span class="pl-k">.</span>convertToGenericDataRecord(
+        json<span class="pl-k">.</span>getBytes(<span class="pl-smi">CharEncoding</span><span class="pl-c1"><span class="pl-k">.</span>UTF_8</span>), <span class="pl-k">new</span> <span class="pl-smi">org.apache.avro<span class="pl-k">.</span>Schema</span>.<span class="pl-smi">Parser</span>()<span class="pl-k">.</span>parse(avroSchema));
+
+    <span class="pl-c"><span class="pl-c">//</span> prepare carbon schema from avro schema </span>
+    <span class="pl-smi">org.apache.carbondata.sdk.file<span class="pl-k">.</span>Schema</span> carbonSchema <span class="pl-k">=</span>
+            <span class="pl-smi">AvroCarbonWriter</span><span class="pl-k">.</span>getCarbonSchemaFromAvroSchema(avroSchema);
+
+    <span class="pl-k">try</span> {
+      <span class="pl-smi">CarbonWriter</span> writer <span class="pl-k">=</span> <span class="pl-smi">CarbonWriter</span><span class="pl-k">.</span>builder()
+          .withSchema(carbonSchema)
+          .outputPath(path)
+          .buildWriterForAvroInput();
+
+      <span class="pl-k">for</span> (<span class="pl-k">int</span> i <span class="pl-k">=</span> <span class="pl-c1">0</span>; i <span class="pl-k">&lt;</span> <span class="pl-c1">100</span>; i<span class="pl-k">++</span>) {
+        writer<span class="pl-k">.</span>write(record);
+      }
+      writer<span class="pl-k">.</span>close();
+    } <span class="pl-k">catch</span> (<span class="pl-smi">Exception</span> e) {
+      e<span class="pl-k">.</span>printStackTrace();
+    }
+  }
+}</pre></div>
+<h2>
+<a id="datatypes-mapping" class="anchor" href="#datatypes-mapping" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Datatypes Mapping</h2>
+<p>Each of SQL data types are mapped into data types of SDK. Following are the mapping:</p>
+<table>
+<thead>
+<tr>
+<th>SQL DataTypes</th>
+<th>Mapped SDK DataTypes</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>BOOLEAN</td>
+<td>DataTypes.BOOLEAN</td>
+</tr>
+<tr>
+<td>SMALLINT</td>
+<td>DataTypes.SHORT</td>
+</tr>
+<tr>
+<td>INTEGER</td>
+<td>DataTypes.INT</td>
+</tr>
+<tr>
+<td>BIGINT</td>
+<td>DataTypes.LONG</td>
+</tr>
+<tr>
+<td>DOUBLE</td>
+<td>DataTypes.DOUBLE</td>
+</tr>
+<tr>
+<td>VARCHAR</td>
+<td>DataTypes.STRING</td>
+</tr>
+<tr>
+<td>DATE</td>
+<td>DataTypes.DATE</td>
+</tr>
+<tr>
+<td>TIMESTAMP</td>
+<td>DataTypes.TIMESTAMP</td>
+</tr>
+<tr>
+<td>STRING</td>
+<td>DataTypes.STRING</td>
+</tr>
+<tr>
+<td>DECIMAL</td>
+<td>DataTypes.createDecimalType(precision, scale)</td>
+</tr>
+</tbody>
+</table>
+<h2>
+<a id="api-list" class="anchor" href="#api-list" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>API List</h2>
+<h3>
+<a id="class-orgapachecarbondatasdkfilecarbonwriterbuilder" class="anchor" href="#class-orgapachecarbondatasdkfilecarbonwriterbuilder" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.CarbonWriterBuilder</h3>
+<pre><code>/**
+* prepares the builder with the schema provided
+* @param schema is instance of Schema
+*        This method must be called when building CarbonWriterBuilder
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder withSchema(Schema schema);
+</code></pre>
+<pre><code>/**
+* Sets the output path of the writer builder
+* @param path is the absolute path where output files are written
+*             This method must be called when building CarbonWriterBuilder
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder outputPath(String path);
+</code></pre>
+<pre><code>/**
+* If set false, writes the carbondata and carbonindex files in a flat folder structure
+* @param isTransactionalTable is a boolelan value
+*             if set to false, then writes the carbondata and carbonindex files
+*                                                            in a flat folder structure.
+*             if set to true, then writes the carbondata and carbonindex files
+*                                                            in segment folder structure..
+*             By default set to false.
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder isTransactionalTable(boolean isTransactionalTable);
+</code></pre>
+<pre><code>/**
+* to set the timestamp in the carbondata and carbonindex index files
+* @param UUID is a timestamp to be used in the carbondata and carbonindex index files.
+*             By default set to zero.
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder uniqueIdentifier(long UUID);
+</code></pre>
+<pre><code>/**
+* To set the carbondata file size in MB between 1MB-2048MB
+* @param blockSize is size in MB between 1MB to 2048 MB
+*                  default value is 1024 MB
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder withBlockSize(int blockSize);
+</code></pre>
+<pre><code>/**
+* To set the blocklet size of carbondata file
+* @param blockletSize is blocklet size in MB
+*                     default value is 64 MB
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder withBlockletSize(int blockletSize);
+</code></pre>
+<pre><code>/**
+* sets the list of columns that needs to be in sorted order
+* @param sortColumns is a string array of columns that needs to be sorted.
+*                    If it is null or by default all dimensions are selected for sorting
+*                    If it is empty array, no columns are sorted
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder sortBy(String[] sortColumns);
+</code></pre>
+<pre><code>/**
+* If set, create a schema file in metadata folder.
+* @param persist is a boolean value, If set to true, creates a schema file in metadata folder.
+*                By default set to false. will not create metadata folder
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder persistSchemaFile(boolean persist);
+</code></pre>
+<pre><code>/**
+* sets the taskNo for the writer. SDKs concurrently running
+* will set taskNo in order to avoid conflicts in file's name during write.
+* @param taskNo is the TaskNo user wants to specify.
+*               by default it is system time in nano seconds.
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder taskNo(String taskNo);
+</code></pre>
+<pre><code>/**
+* To support the load options for sdk writer
+* @param options key,value pair of load options.
+*                supported keys values are
+*                a. bad_records_logger_enable -- true (write into separate logs), false
+*                b. bad_records_action -- FAIL, FORCE, IGNORE, REDIRECT
+*                c. bad_record_path -- path
+*                d. dateformat -- same as JAVA SimpleDateFormat
+*                e. timestampformat -- same as JAVA SimpleDateFormat
+*                f. complex_delimiter_level_1 -- value to Split the complexTypeData
+*                g. complex_delimiter_level_2 -- value to Split the nested complexTypeData
+*                h. quotechar
+*                i. escapechar
+*
+*                Default values are as follows.
+*
+*                a. bad_records_logger_enable -- "false"
+*                b. bad_records_action -- "FAIL"
+*                c. bad_record_path -- ""
+*                d. dateformat -- "" , uses from carbon.properties file
+*                e. timestampformat -- "", uses from carbon.properties file
+*                f. complex_delimiter_level_1 -- "$"
+*                g. complex_delimiter_level_2 -- ":"
+*                h. quotechar -- "\""
+*                i. escapechar -- "\\"
+*
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder withLoadOptions(Map&lt;String, String&gt; options);
+</code></pre>
+<pre><code>/**
+* Build a {@link CarbonWriter}, which accepts row in CSV format object
+* @return CSVCarbonWriter
+* @throws IOException
+* @throws InvalidLoadOptionException
+*/
+public CarbonWriter buildWriterForCSVInput() throws IOException, InvalidLoadOptionException;
+</code></pre>
+<pre><code>/**
+* Build a {@link CarbonWriter}, which accepts Avro format object
+* @return AvroCarbonWriter 
+* @throws IOException
+* @throws InvalidLoadOptionException
+*/
+public CarbonWriter buildWriterForAvroInput() throws IOException, InvalidLoadOptionException;
+</code></pre>
+<h3>
+<a id="class-orgapachecarbondatasdkfilecarbonwriter" class="anchor" href="#class-orgapachecarbondatasdkfilecarbonwriter" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.CarbonWriter</h3>
+<pre><code>/**
+* Write an object to the file, the format of the object depends on the implementation
+* If AvroCarbonWriter, object is of type org.apache.avro.generic.GenericData.Record 
+* If CSVCarbonWriter, object is of type String[]
+* Note: This API is not thread safe
+* @param object
+* @throws IOException
+*/
+public abstract void write(Object object) throws IOException;
+</code></pre>
+<pre><code>/**
+* Flush and close the writer
+*/
+public abstract void close() throws IOException;
+</code></pre>
+<pre><code>/**
+* Create a {@link CarbonWriterBuilder} to build a {@link CarbonWriter}
+*/
+public static CarbonWriterBuilder builder() {
+return new CarbonWriterBuilder();
+}
+</code></pre>
+<h3>
+<a id="class-orgapachecarbondatasdkfilefield" class="anchor" href="#class-orgapachecarbondatasdkfilefield" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.Field</h3>
+<pre><code>/**
+* Field Constructor
+* @param name name of the field
+* @param type datatype of field, specified in strings.
+*/
+public Field(String name, String type);
+</code></pre>
+<pre><code>/**
+* Field constructor
+* @param name name of the field
+* @param type datatype of the field of class DataType
+*/
+public Field(String name, DataType type);  
+</code></pre>
+<h3>
+<a id="class-orgapachecarbondatasdkfileschema" class="anchor" href="#class-orgapachecarbondatasdkfileschema" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.Schema</h3>
+<pre><code>/**
+* construct a schema with fields
+* @param fields
+*/
+public Schema(Field[] fields);
+</code></pre>
+<pre><code>/**
+* Create a Schema using JSON string, for example:
+* [
+*   {"name":"string"},
+*   {"age":"int"}
+* ] 
+* @param json specified as string
+* @return Schema
+*/
+public static Schema parseJson(String json);
+</code></pre>
+<h3>
+<a id="class-orgapachecarbondatasdkfileavrocarbonwriter" class="anchor" href="#class-orgapachecarbondatasdkfileavrocarbonwriter" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.AvroCarbonWriter</h3>
+<pre><code>/**
+* converts avro schema to carbon schema, required by carbonWriter
+*
+* @param avroSchemaString json formatted avro schema as string
+* @return carbon sdk schema
+*/
+public static org.apache.carbondata.sdk.file.Schema getCarbonSchemaFromAvroSchema(String avroSchemaString);
+</code></pre>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/bee56334/content/troubleshooting.html
----------------------------------------------------------------------
diff --git a/content/troubleshooting.html b/content/troubleshooting.html
new file mode 100644
index 0000000..c668dc9
--- /dev/null
+++ b/content/troubleshooting.html
@@ -0,0 +1,366 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="row">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="troubleshooting" class="anchor" href="#troubleshooting" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Troubleshooting</h1>
+<p>This tutorial is designed to provide troubleshooting for end users and developers
+who are building, deploying, and using CarbonData.</p>
+<h2>
+<a id="when-loading-data-gets-tablestatuslock-issues" class="anchor" href="#when-loading-data-gets-tablestatuslock-issues" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>When loading data, gets tablestatus.lock issues:</h2>
+<p><strong>Symptom</strong></p>
+<pre><code>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.&lt;init&gt;(FileOutputStream.java:213)
+	at java.io.FileOutputStream.&lt;init&gt;(FileOutputStream.java:101)
+</code></pre>
+<p><strong>Possible Cause</strong>
+If you use <code>&lt;hdfs path&gt;</code> as store path when creating carbonsession, may get the errors,because the default is LOCALLOCK.</p>
+<p><strong>Procedure</strong>
+Before creating carbonsession, sets as below:</p>
+<pre><code>import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+CarbonProperties.getInstance().addProperty(CarbonCommonConstants.LOCK_TYPE, "HDFSLOCK")
+</code></pre>
+<h2>
+<a id="failed-to-load-thrift-libraries" class="anchor" href="#failed-to-load-thrift-libraries" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to load thrift libraries</h2>
+<p><strong>Symptom</strong></p>
+<p>Thrift throws following exception :</p>
+<pre><code>thrift: error while loading shared libraries:
+libthriftc.so.0: cannot open shared object file: No such file or directory
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The complete path to the directory containing the libraries is not configured correctly.</p>
+<p><strong>Procedure</strong></p>
+<p>Follow the Apache thrift docs at <a href="https://thrift.apache.org/docs/install" target=_blank rel="nofollow">https://thrift.apache.org/docs/install</a> to install thrift correctly.</p>
+<h2>
+<a id="failed-to-launch-the-spark-shell" class="anchor" href="#failed-to-launch-the-spark-shell" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to launch the Spark Shell</h2>
+<p><strong>Symptom</strong></p>
+<p>The shell prompts the following error :</p>
+<pre><code>org.apache.spark.sql.CarbonContext$$anon$$apache$spark$sql$catalyst$analysis
+$OverrideCatalog$_setter_$org$apache$spark$sql$catalyst$analysis
+$OverrideCatalog$$overrides_$e
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The Spark Version and the selected Spark Profile do not match.</p>
+<p><strong>Procedure</strong></p>
+<ol>
+<li>
+<p>Ensure your spark version and selected profile for spark are correct.</p>
+</li>
+<li>
+<p>Use the following command :</p>
+</li>
+</ol>
+<pre><code>"mvn -Pspark-2.1 -Dspark.version {yourSparkVersion} clean package"
+</code></pre>
+<p>Note :  Refrain from using "mvn clean package" without specifying the profile.</p>
+<h2>
+<a id="failed-to-execute-load-query-on-cluster" class="anchor" href="#failed-to-execute-load-query-on-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute load query on cluster.</h2>
+<p><strong>Symptom</strong></p>
+<p>Load query failed with the following exception:</p>
+<pre><code>Dictionary file is locked for updation.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The carbon.properties file is not identical in all the nodes of the cluster.</p>
+<p><strong>Procedure</strong></p>
+<p>Follow the steps to ensure the carbon.properties file is consistent across all the nodes:</p>
+<ol>
+<li>
+<p>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.</p>
+</li>
+<li>
+<p>For the changes to take effect, restart the Spark cluster.</p>
+</li>
+</ol>
+<h2>
+<a id="failed-to-execute-insert-query-on-cluster" class="anchor" href="#failed-to-execute-insert-query-on-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute insert query on cluster.</h2>
+<p><strong>Symptom</strong></p>
+<p>Load query failed with the following exception:</p>
+<pre><code>Dictionary file is locked for updation.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The carbon.properties file is not identical in all the nodes of the cluster.</p>
+<p><strong>Procedure</strong></p>
+<p>Follow the steps to ensure the carbon.properties file is consistent across all the nodes:</p>
+<ol>
+<li>
+<p>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.</p>
+</li>
+<li>
+<p>For the changes to take effect, restart the Spark cluster.</p>
+</li>
+</ol>
+<h2>
+<a id="failed-to-connect-to-hiveuser-with-thrift" class="anchor" href="#failed-to-connect-to-hiveuser-with-thrift" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to connect to hiveuser with thrift</h2>
+<p><strong>Symptom</strong></p>
+<p>We get the following exception :</p>
+<pre><code>Cannot connect to hiveuser.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The external process does not have permission to access.</p>
+<p><strong>Procedure</strong></p>
+<p>Ensure that the Hiveuser in mysql must allow its access to the external processes.</p>
+<h2>
+<a id="failed-to-read-the-metastore-db-during-table-creation" class="anchor" href="#failed-to-read-the-metastore-db-during-table-creation" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to read the metastore db during table creation.</h2>
+<p><strong>Symptom</strong></p>
+<p>We get the following exception on trying to connect :</p>
+<pre><code>Cannot read the metastore db
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The metastore db is dysfunctional.</p>
+<p><strong>Procedure</strong></p>
+<p>Remove the metastore db from the carbon.metastore in the Spark Directory.</p>
+<h2>
+<a id="failed-to-load-data-on-the-cluster" class="anchor" href="#failed-to-load-data-on-the-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to load data on the cluster</h2>
+<p><strong>Symptom</strong></p>
+<p>Data loading fails with the following exception :</p>
+<pre><code>Data Load failure exception
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The following issue can cause the failure :</p>
+<ol>
+<li>
+<p>The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.</p>
+</li>
+<li>
+<p>Path to hdfs ddl is not configured correctly in the carbon.properties.</p>
+</li>
+</ol>
+<p><strong>Procedure</strong></p>
+<p>Follow the steps to ensure the following configuration files are consistent across all the nodes:</p>
+<ol>
+<li>
+<p>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.</p>
+<p>Note : Set the path to hdfs ddl in carbon.properties in the master node.</p>
+</li>
+<li>
+<p>For the changes to take effect, restart the Spark cluster.</p>
+</li>
+</ol>
+<h2>
+<a id="failed-to-insert-data-on-the-cluster" class="anchor" href="#failed-to-insert-data-on-the-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to insert data on the cluster</h2>
+<p><strong>Symptom</strong></p>
+<p>Insertion fails with the following exception :</p>
+<pre><code>Data Load failure exception
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The following issue can cause the failure :</p>
+<ol>
+<li>
+<p>The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.</p>
+</li>
+<li>
+<p>Path to hdfs ddl is not configured correctly in the carbon.properties.</p>
+</li>
+</ol>
+<p><strong>Procedure</strong></p>
+<p>Follow the steps to ensure the following configuration files are consistent across all the nodes:</p>
+<ol>
+<li>
+<p>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.</p>
+<p>Note : Set the path to hdfs ddl in carbon.properties in the master node.</p>
+</li>
+<li>
+<p>For the changes to take effect, restart the Spark cluster.</p>
+</li>
+</ol>
+<h2>
+<a id="failed-to-execute-concurrent-operationsloadinsertupdate-on-table-by-multiple-workers" class="anchor" href="#failed-to-execute-concurrent-operationsloadinsertupdate-on-table-by-multiple-workers" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute Concurrent Operations(Load,Insert,Update) on table by multiple workers.</h2>
+<p><strong>Symptom</strong></p>
+<p>Execution fails with the following exception :</p>
+<pre><code>Table is locked for updation.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>Concurrency not supported.</p>
+<p><strong>Procedure</strong></p>
+<p>Worker must wait for the query execution to complete and the table to release the lock for another query execution to succeed.</p>
+<h2>
+<a id="failed-to-create-a-table-with-a-single-numeric-column" class="anchor" href="#failed-to-create-a-table-with-a-single-numeric-column" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to create a table with a single numeric column.</h2>
+<p><strong>Symptom</strong></p>
+<p>Execution fails with the following exception :</p>
+<pre><code>Table creation fails.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>Behaviour not supported.</p>
+<p><strong>Procedure</strong></p>
+<p>A single column that can be considered as dimension is mandatory for table creation.</p>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file


[23/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/quick-start-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/quick-start-guide.html b/src/main/webapp/quick-start-guide.html
index ea88086..f3e8a8f 100644
--- a/src/main/webapp/quick-start-guide.html
+++ b/src/main/webapp/quick-start-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -217,10 +217,10 @@
 <a id="prerequisites" class="anchor" href="#prerequisites" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h2>
 <ul>
 <li>
-<p>Spark 2.2.1 version is installed and running.CarbonData supports Spark versions upto 2.2.1.Please follow steps described in <a href="https://spark.apache.org/docs/latest" target=_blank rel="nofollow">Spark docs website</a> for installing and running Spark.</p>
+<p>CarbonData supports Spark versions upto 2.2.1.Please download Spark package from <a href="https://spark.apache.org/downloads.html" target=_blank rel="nofollow">Spark website</a></p>
 </li>
 <li>
-<p>Create a sample.csv file using the following commands. The CSV file is required for loading data into CarbonData.</p>
+<p>Create a sample.csv file using the following commands. The CSV file is required for loading data into CarbonData</p>
 <pre><code>cd carbondata
 cat &gt; sample.csv &lt;&lt; EOF
 id,name,city,age
@@ -232,21 +232,18 @@ EOF
 </li>
 </ul>
 <h2>
-<a id="deployment-modes" class="anchor" href="#deployment-modes" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deployment modes</h2>
+<a id="integration" class="anchor" href="#integration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Integration</h2>
 <p>CarbonData can be integrated with Spark and Presto Execution Engines.The below documentation guides on Installing and Configuring with these execution engines.</p>
 <h3>
 <a id="spark" class="anchor" href="#spark" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Spark</h3>
 <p><a href="#installing-and-configuring-carbondata-to-run-locally-with-spark-shell">Installing and Configuring CarbonData to run locally with Spark Shell</a></p>
 <p><a href="#installing-and-configuring-carbondata-on-standalone-spark-cluster">Installing and Configuring CarbonData on Standalone Spark Cluster</a></p>
 <p><a href="#installing-and-configuring-carbondata-on-spark-on-yarn-cluster">Installing and Configuring CarbonData on Spark on YARN Cluster</a></p>
+<p><a href="#query-execution-using-carbondata-thrift-server">Installing and Configuring CarbonData Thrift Server for Query Execution</a></p>
 <h3>
 <a id="presto" class="anchor" href="#presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Presto</h3>
 <p><a href="#installing-and-configuring-carbondata-on-presto">Installing and Configuring CarbonData on Presto</a></p>
 <h2>
-<a id="querying-data" class="anchor" href="#querying-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Querying Data</h2>
-<p><a href="#query-execution-using-carbondata-thrift-server">Query Execution using CarbonData Thrift Server</a></p>
-<h2></h2>
-<h2>
 <a id="installing-and-configuring-carbondata-to-run-locally-with-spark-shell" class="anchor" href="#installing-and-configuring-carbondata-to-run-locally-with-spark-shell" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData to run locally with Spark Shell</h2>
 <p>Apache Spark Shell provides a simple way to learn the API, as well as a powerful tool to analyze data interactively. Please visit <a href="http://spark.apache.org/docs/latest/" target=_blank rel="nofollow">Apache Spark Documentation</a> for more details on Spark shell.</p>
 <h4>
@@ -275,12 +272,12 @@ import org.apache.spark.sql.CarbonSession._
 <h6>
 <a id="creating-a-table" class="anchor" href="#creating-a-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Creating a Table</h6>
 <pre><code>scala&gt;carbon.sql("CREATE TABLE
-                        IF NOT EXISTS test_table(
-                                  id string,
-                                  name string,
-                                  city string,
-                                  age Int)
-                       STORED BY 'carbondata'")
+                    IF NOT EXISTS test_table(
+                    id string,
+                    name string,
+                    city string,
+                    age Int)
+                  STORED AS carbondata")
 </code></pre>
 <h6>
 <a id="loading-data-to-a-table" class="anchor" href="#loading-data-to-a-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Loading Data to a Table</h6>
@@ -558,12 +555,11 @@ hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
 </code></pre>
 <h2>
 <a id="installing-and-configuring-carbondata-on-presto" class="anchor" href="#installing-and-configuring-carbondata-on-presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Presto</h2>
-<ul>
-<li>
+<p><strong>NOTE:</strong> <strong>CarbonData tables cannot be created nor loaded from Presto.User need to create CarbonData Table and load data into it
+either with <a href="#installing-and-configuring-carbondata-to-run-locally-with-spark-shell">Spark</a> or <a href="./sdk-guide.html">SDK</a>.
+Once the table is created,it can be queried from Presto.</strong></p>
 <h3>
 <a id="installing-presto" class="anchor" href="#installing-presto" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing Presto</h3>
-</li>
-</ul>
 <ol>
 <li>
 <p>Download the 0.187 version of Presto using:
@@ -684,12 +680,23 @@ discovery.uri=&lt;coordinator_ip&gt;:8086
 <pre><code>select * from system.runtime.nodes;
 </code></pre>
 <p>Now you can use the Presto CLI on the coordinator to query data sources in the catalog using the Presto workers.</p>
+<p>List the schemas(databases) available</p>
+<pre><code>show schemas;
+</code></pre>
+<p>Selected the schema where CarbonData table resides</p>
+<pre><code>use carbonschema;
+</code></pre>
+<p>List the available tables</p>
+<pre><code>show tables;
+</code></pre>
+<p>Query from the available tables</p>
+<pre><code>select * from carbon_table;
+</code></pre>
 <p><strong>Note :</strong> Create Tables and data loads should be done before executing queries as we can not create carbon table from this interface.</p>
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__quickstart').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -704,4 +711,4 @@ $(function() { $('.b-nav__quickstart').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/release-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/release-guide.html b/src/main/webapp/release-guide.html
index fb51368..cb47540 100644
--- a/src/main/webapp/release-guide.html
+++ b/src/main/webapp/release-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -553,8 +553,7 @@ contributions.</p>
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__release').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -569,4 +568,4 @@ $(function() { $('.b-nav__release').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/s3-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/s3-guide.html b/src/main/webapp/s3-guide.html
index 9042b19..57af913 100644
--- a/src/main/webapp/s3-guide.html
+++ b/src/main/webapp/s3-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -277,8 +277,7 @@ to a HDFS directory.</li>
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__s3').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -293,4 +292,4 @@ $(function() { $('.b-nav__s3').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/sdk-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/sdk-guide.html b/src/main/webapp/sdk-guide.html
index 95ae214..a252965 100644
--- a/src/main/webapp/sdk-guide.html
+++ b/src/main/webapp/sdk-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -212,9 +212,14 @@
                                     <div>
 <h1>
 <a id="sdk-guide" class="anchor" href="#sdk-guide" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SDK Guide</h1>
-<p>In the carbon jars package, there exist a carbondata-store-sdk-x.x.x-SNAPSHOT.jar, including SDK writer and reader.</p>
+<p>CarbonData provides SDK to facilitate</p>
+<ol>
+<li><a href="#sdk-writer">Writing carbondata files from other application which does not use Spark</a></li>
+<li><a href="#sdk-reader">Reading carbondata files from other application which does not use Spark</a></li>
+</ol>
 <h1>
 <a id="sdk-writer" class="anchor" href="#sdk-writer" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SDK Writer</h1>
+<p>In the carbon jars package, there exist a carbondata-store-sdk-x.x.x-SNAPSHOT.jar, including SDK writer and reader.</p>
 <p>This SDK writer, writes carbondata file and carbonindex file at a given path.
 External client can make use of this writer to convert other format data or live data to create carbondata and index files.
 These SDK writer output contains just a carbondata and carbonindex files. No metadata folder will be present.</p>
@@ -989,8 +994,7 @@ public String getProperty(String key, String defaultValue);
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__api').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -1005,4 +1009,4 @@ $(function() { $('.b-nav__api').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/sdk-writer-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/sdk-writer-guide.html b/src/main/webapp/sdk-writer-guide.html
deleted file mode 100644
index 5553194..0000000
--- a/src/main/webapp/sdk-writer-guide.html
+++ /dev/null
@@ -1,549 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
-                                   target="_blank">Apache CarbonData 1.4.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
-                                   target="_blank">Apache CarbonData 1.3.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
-                                   target="_blank">Apache CarbonData 1.3.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
-                                   target="_blank">Apache CarbonData 1.2.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
-                                   target="_blank">Apache CarbonData 1.1.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
-                                   target="_blank">Apache CarbonData 1.1.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
-                                   target="_blank">Apache CarbonData 1.0.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
-                                   target="_blank">Apache CarbonData 0.2.0</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
-                                   target="_blank">Apache CarbonData 0.1.1</a></li>
-                            <li>
-                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
-                                   target="_blank">Apache CarbonData 0.1.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search" target="_blank">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input"  placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="row">
-                                <div class="col-sm-12  col-md-12">
-                                    <div><h1>
-<a id="sdk-writer-guide" class="anchor" href="#sdk-writer-guide" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SDK Writer Guide</h1>
-<p>In the carbon jars package, there exist a carbondata-store-sdk-x.x.x-SNAPSHOT.jar.
-This SDK writer, writes carbondata file and carbonindex file at a given path.
-External client can make use of this writer to convert other format data or live data to create carbondata and index files.
-These SDK writer output contains just a carbondata and carbonindex files. No metadata folder will be present.</p>
-<h2>
-<a id="quick-example" class="anchor" href="#quick-example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Quick example</h2>
-<h3>
-<a id="example-with-csv-format" class="anchor" href="#example-with-csv-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example with csv format</h3>
-<div class="highlight highlight-source-java"><pre> <span class="pl-k">import</span> <span class="pl-smi">java.io.IOException</span>;
- 
- <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException</span>;
- <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.core.metadata.datatype.DataTypes</span>;
- <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.CarbonWriter</span>;
- <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.CarbonWriterBuilder</span>;
- <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.Field</span>;
- <span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.Schema</span>;
- 
- <span class="pl-k">public</span> <span class="pl-k">class</span> <span class="pl-en">TestSdk</span> {
- 
-   <span class="pl-k">public</span> <span class="pl-k">static</span> <span class="pl-k">void</span> <span class="pl-en">main</span>(<span class="pl-k">String</span>[] <span class="pl-v">args</span>) <span class="pl-k">throws</span> <span class="pl-smi">IOException</span>, <span class="pl-smi">InvalidLoadOptionException</span> {
-     testSdkWriter();
-   }
- 
-   <span class="pl-k">public</span> <span class="pl-k">static</span> <span class="pl-k">void</span> <span class="pl-en">testSdkWriter</span>() <span class="pl-k">throws</span> <span class="pl-smi">IOException</span>, <span class="pl-smi">InvalidLoadOptionException</span> {
-     <span class="pl-smi">String</span> path <span class="pl-k">=</span> <span class="pl-s"><span class="pl-pds">"</span>/home/root1/Documents/ab/temp<span class="pl-pds">"</span></span>;
- 
-     <span class="pl-k">Field</span>[] fields <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">Field</span>[<span class="pl-c1">2</span>];
-     fields[<span class="pl-c1">0</span>] <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">Field</span>(<span class="pl-s"><span class="pl-pds">"</span>name<span class="pl-pds">"</span></span>, <span class="pl-smi">DataTypes</span><span class="pl-c1"><span class="pl-k">.</span>STRING</span>);
-     fields[<span class="pl-c1">1</span>] <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">Field</span>(<span class="pl-s"><span class="pl-pds">"</span>age<span class="pl-pds">"</span></span>, <span class="pl-smi">DataTypes</span><span class="pl-c1"><span class="pl-k">.</span>INT</span>);
- 
-     <span class="pl-smi">Schema</span> schema <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">Schema</span>(fields);
- 
-     <span class="pl-smi">CarbonWriterBuilder</span> builder <span class="pl-k">=</span> <span class="pl-smi">CarbonWriter</span><span class="pl-k">.</span>builder()<span class="pl-k">.</span>withSchema(schema)<span class="pl-k">.</span>outputPath(path);
- 
-     <span class="pl-smi">CarbonWriter</span> writer <span class="pl-k">=</span> builder<span class="pl-k">.</span>buildWriterForCSVInput();
- 
-     <span class="pl-k">int</span> rows <span class="pl-k">=</span> <span class="pl-c1">5</span>;
-     <span class="pl-k">for</span> (<span class="pl-k">int</span> i <span class="pl-k">=</span> <span class="pl-c1">0</span>; i <span class="pl-k">&lt;</span> rows; i<span class="pl-k">++</span>) {
-       writer<span class="pl-k">.</span>write(<span class="pl-k">new</span> <span class="pl-smi">String</span>[] { <span class="pl-s"><span class="pl-pds">"</span>robot<span class="pl-pds">"</span></span> <span class="pl-k">+</span> (i <span class="pl-k">%</span> <span class="pl-c1">10</span>), <span class="pl-smi">String</span><span class="pl-k">.</span>valueOf(i) });
-     }
-     writer<span class="pl-k">.</span>close();
-   }
- }</pre></div>
-<h3>
-<a id="example-with-avro-format" class="anchor" href="#example-with-avro-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Example with Avro format</h3>
-<div class="highlight highlight-source-java"><pre><span class="pl-k">import</span> <span class="pl-smi">java.io.IOException</span>;
-
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException</span>;
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.core.metadata.datatype.DataTypes</span>;
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.AvroCarbonWriter</span>;
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.CarbonWriter</span>;
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.carbondata.sdk.file.Field</span>;
-
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.avro.generic.GenericData</span>;
-<span class="pl-k">import</span> <span class="pl-smi">org.apache.commons.lang.CharEncoding</span>;
-
-<span class="pl-k">import</span> <span class="pl-smi">tech.allegro.schema.json2avro.converter.JsonAvroConverter</span>;
-
-<span class="pl-k">public</span> <span class="pl-k">class</span> <span class="pl-en">TestSdkAvro</span> {
-
-  <span class="pl-k">public</span> <span class="pl-k">static</span> <span class="pl-k">void</span> <span class="pl-en">main</span>(<span class="pl-k">String</span>[] <span class="pl-v">args</span>) <span class="pl-k">throws</span> <span class="pl-smi">IOException</span>, <span class="pl-smi">InvalidLoadOptionException</span> {
-    testSdkWriter();
-  }
-
-
-  <span class="pl-k">public</span> <span class="pl-k">static</span> <span class="pl-k">void</span> <span class="pl-en">testSdkWriter</span>() <span class="pl-k">throws</span> <span class="pl-smi">IOException</span>, <span class="pl-smi">InvalidLoadOptionException</span> {
-    <span class="pl-smi">String</span> path <span class="pl-k">=</span> <span class="pl-s"><span class="pl-pds">"</span>./AvroCarbonWriterSuiteWriteFiles<span class="pl-pds">"</span></span>;
-    <span class="pl-c"><span class="pl-c">//</span> Avro schema</span>
-    <span class="pl-smi">String</span> avroSchema <span class="pl-k">=</span>
-        <span class="pl-s"><span class="pl-pds">"</span>{<span class="pl-pds">"</span></span> <span class="pl-k">+</span>
-            <span class="pl-s"><span class="pl-pds">"</span>   <span class="pl-cce">\"</span>type<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>record<span class="pl-cce">\"</span>,<span class="pl-pds">"</span></span> <span class="pl-k">+</span>
-            <span class="pl-s"><span class="pl-pds">"</span>   <span class="pl-cce">\"</span>name<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>Acme<span class="pl-cce">\"</span>,<span class="pl-pds">"</span></span> <span class="pl-k">+</span>
-            <span class="pl-s"><span class="pl-pds">"</span>   <span class="pl-cce">\"</span>fields<span class="pl-cce">\"</span> : [<span class="pl-pds">"</span></span>
-            <span class="pl-k">+</span> <span class="pl-s"><span class="pl-pds">"</span>{ <span class="pl-cce">\"</span>name<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>fname<span class="pl-cce">\"</span>, <span class="pl-cce">\"</span>type<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>string<span class="pl-cce">\"</span> },<span class="pl-pds">"</span></span>
-            <span class="pl-k">+</span> <span class="pl-s"><span class="pl-pds">"</span>{ <span class="pl-cce">\"</span>name<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>age<span class="pl-cce">\"</span>, <span class="pl-cce">\"</span>type<span class="pl-cce">\"</span> : <span class="pl-cce">\"</span>int<span class="pl-cce">\"</span> }]<span class="pl-pds">"</span></span> <span class="pl-k">+</span>
-            <span class="pl-s"><span class="pl-pds">"</span>}<span class="pl-pds">"</span></span>;
-
-    <span class="pl-smi">String</span> json <span class="pl-k">=</span> <span class="pl-s"><span class="pl-pds">"</span>{<span class="pl-cce">\"</span>fname<span class="pl-cce">\"</span>:<span class="pl-cce">\"</span>bob<span class="pl-cce">\"</span>, <span class="pl-cce">\"</span>age<span class="pl-cce">\"</span>:10}<span class="pl-pds">"</span></span>;
-
-    <span class="pl-c"><span class="pl-c">//</span> conversion to GenericData.Record</span>
-    <span class="pl-smi">JsonAvroConverter</span> converter <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-smi">JsonAvroConverter</span>();
-    <span class="pl-smi">GenericData</span><span class="pl-k">.</span><span class="pl-smi">Record</span> record <span class="pl-k">=</span> converter<span class="pl-k">.</span>convertToGenericDataRecord(
-        json<span class="pl-k">.</span>getBytes(<span class="pl-smi">CharEncoding</span><span class="pl-c1"><span class="pl-k">.</span>UTF_8</span>), <span class="pl-k">new</span> <span class="pl-smi">org.apache.avro<span class="pl-k">.</span>Schema</span>.<span class="pl-smi">Parser</span>()<span class="pl-k">.</span>parse(avroSchema));
-
-    <span class="pl-c"><span class="pl-c">//</span> prepare carbon schema from avro schema </span>
-    <span class="pl-smi">org.apache.carbondata.sdk.file<span class="pl-k">.</span>Schema</span> carbonSchema <span class="pl-k">=</span>
-            <span class="pl-smi">AvroCarbonWriter</span><span class="pl-k">.</span>getCarbonSchemaFromAvroSchema(avroSchema);
-
-    <span class="pl-k">try</span> {
-      <span class="pl-smi">CarbonWriter</span> writer <span class="pl-k">=</span> <span class="pl-smi">CarbonWriter</span><span class="pl-k">.</span>builder()
-          .withSchema(carbonSchema)
-          .outputPath(path)
-          .buildWriterForAvroInput();
-
-      <span class="pl-k">for</span> (<span class="pl-k">int</span> i <span class="pl-k">=</span> <span class="pl-c1">0</span>; i <span class="pl-k">&lt;</span> <span class="pl-c1">100</span>; i<span class="pl-k">++</span>) {
-        writer<span class="pl-k">.</span>write(record);
-      }
-      writer<span class="pl-k">.</span>close();
-    } <span class="pl-k">catch</span> (<span class="pl-smi">Exception</span> e) {
-      e<span class="pl-k">.</span>printStackTrace();
-    }
-  }
-}</pre></div>
-<h2>
-<a id="datatypes-mapping" class="anchor" href="#datatypes-mapping" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Datatypes Mapping</h2>
-<p>Each of SQL data types are mapped into data types of SDK. Following are the mapping:</p>
-<table>
-<thead>
-<tr>
-<th>SQL DataTypes</th>
-<th>Mapped SDK DataTypes</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>BOOLEAN</td>
-<td>DataTypes.BOOLEAN</td>
-</tr>
-<tr>
-<td>SMALLINT</td>
-<td>DataTypes.SHORT</td>
-</tr>
-<tr>
-<td>INTEGER</td>
-<td>DataTypes.INT</td>
-</tr>
-<tr>
-<td>BIGINT</td>
-<td>DataTypes.LONG</td>
-</tr>
-<tr>
-<td>DOUBLE</td>
-<td>DataTypes.DOUBLE</td>
-</tr>
-<tr>
-<td>VARCHAR</td>
-<td>DataTypes.STRING</td>
-</tr>
-<tr>
-<td>DATE</td>
-<td>DataTypes.DATE</td>
-</tr>
-<tr>
-<td>TIMESTAMP</td>
-<td>DataTypes.TIMESTAMP</td>
-</tr>
-<tr>
-<td>STRING</td>
-<td>DataTypes.STRING</td>
-</tr>
-<tr>
-<td>DECIMAL</td>
-<td>DataTypes.createDecimalType(precision, scale)</td>
-</tr>
-</tbody>
-</table>
-<h2>
-<a id="api-list" class="anchor" href="#api-list" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>API List</h2>
-<h3>
-<a id="class-orgapachecarbondatasdkfilecarbonwriterbuilder" class="anchor" href="#class-orgapachecarbondatasdkfilecarbonwriterbuilder" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.CarbonWriterBuilder</h3>
-<pre><code>/**
-* prepares the builder with the schema provided
-* @param schema is instance of Schema
-*        This method must be called when building CarbonWriterBuilder
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder withSchema(Schema schema);
-</code></pre>
-<pre><code>/**
-* Sets the output path of the writer builder
-* @param path is the absolute path where output files are written
-*             This method must be called when building CarbonWriterBuilder
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder outputPath(String path);
-</code></pre>
-<pre><code>/**
-* If set false, writes the carbondata and carbonindex files in a flat folder structure
-* @param isTransactionalTable is a boolelan value
-*             if set to false, then writes the carbondata and carbonindex files
-*                                                            in a flat folder structure.
-*             if set to true, then writes the carbondata and carbonindex files
-*                                                            in segment folder structure..
-*             By default set to false.
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder isTransactionalTable(boolean isTransactionalTable);
-</code></pre>
-<pre><code>/**
-* to set the timestamp in the carbondata and carbonindex index files
-* @param UUID is a timestamp to be used in the carbondata and carbonindex index files.
-*             By default set to zero.
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder uniqueIdentifier(long UUID);
-</code></pre>
-<pre><code>/**
-* To set the carbondata file size in MB between 1MB-2048MB
-* @param blockSize is size in MB between 1MB to 2048 MB
-*                  default value is 1024 MB
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder withBlockSize(int blockSize);
-</code></pre>
-<pre><code>/**
-* To set the blocklet size of carbondata file
-* @param blockletSize is blocklet size in MB
-*                     default value is 64 MB
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder withBlockletSize(int blockletSize);
-</code></pre>
-<pre><code>/**
-* sets the list of columns that needs to be in sorted order
-* @param sortColumns is a string array of columns that needs to be sorted.
-*                    If it is null or by default all dimensions are selected for sorting
-*                    If it is empty array, no columns are sorted
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder sortBy(String[] sortColumns);
-</code></pre>
-<pre><code>/**
-* If set, create a schema file in metadata folder.
-* @param persist is a boolean value, If set to true, creates a schema file in metadata folder.
-*                By default set to false. will not create metadata folder
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder persistSchemaFile(boolean persist);
-</code></pre>
-<pre><code>/**
-* sets the taskNo for the writer. SDKs concurrently running
-* will set taskNo in order to avoid conflicts in file's name during write.
-* @param taskNo is the TaskNo user wants to specify.
-*               by default it is system time in nano seconds.
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder taskNo(String taskNo);
-</code></pre>
-<pre><code>/**
-* To support the load options for sdk writer
-* @param options key,value pair of load options.
-*                supported keys values are
-*                a. bad_records_logger_enable -- true (write into separate logs), false
-*                b. bad_records_action -- FAIL, FORCE, IGNORE, REDIRECT
-*                c. bad_record_path -- path
-*                d. dateformat -- same as JAVA SimpleDateFormat
-*                e. timestampformat -- same as JAVA SimpleDateFormat
-*                f. complex_delimiter_level_1 -- value to Split the complexTypeData
-*                g. complex_delimiter_level_2 -- value to Split the nested complexTypeData
-*                h. quotechar
-*                i. escapechar
-*
-*                Default values are as follows.
-*
-*                a. bad_records_logger_enable -- "false"
-*                b. bad_records_action -- "FAIL"
-*                c. bad_record_path -- ""
-*                d. dateformat -- "" , uses from carbon.properties file
-*                e. timestampformat -- "", uses from carbon.properties file
-*                f. complex_delimiter_level_1 -- "$"
-*                g. complex_delimiter_level_2 -- ":"
-*                h. quotechar -- "\""
-*                i. escapechar -- "\\"
-*
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder withLoadOptions(Map&lt;String, String&gt; options);
-</code></pre>
-<pre><code>/**
-* Build a {@link CarbonWriter}, which accepts row in CSV format object
-* @return CSVCarbonWriter
-* @throws IOException
-* @throws InvalidLoadOptionException
-*/
-public CarbonWriter buildWriterForCSVInput() throws IOException, InvalidLoadOptionException;
-</code></pre>
-<pre><code>/**
-* Build a {@link CarbonWriter}, which accepts Avro format object
-* @return AvroCarbonWriter 
-* @throws IOException
-* @throws InvalidLoadOptionException
-*/
-public CarbonWriter buildWriterForAvroInput() throws IOException, InvalidLoadOptionException;
-</code></pre>
-<h3>
-<a id="class-orgapachecarbondatasdkfilecarbonwriter" class="anchor" href="#class-orgapachecarbondatasdkfilecarbonwriter" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.CarbonWriter</h3>
-<pre><code>/**
-* Write an object to the file, the format of the object depends on the implementation
-* If AvroCarbonWriter, object is of type org.apache.avro.generic.GenericData.Record 
-* If CSVCarbonWriter, object is of type String[]
-* Note: This API is not thread safe
-* @param object
-* @throws IOException
-*/
-public abstract void write(Object object) throws IOException;
-</code></pre>
-<pre><code>/**
-* Flush and close the writer
-*/
-public abstract void close() throws IOException;
-</code></pre>
-<pre><code>/**
-* Create a {@link CarbonWriterBuilder} to build a {@link CarbonWriter}
-*/
-public static CarbonWriterBuilder builder() {
-return new CarbonWriterBuilder();
-}
-</code></pre>
-<h3>
-<a id="class-orgapachecarbondatasdkfilefield" class="anchor" href="#class-orgapachecarbondatasdkfilefield" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.Field</h3>
-<pre><code>/**
-* Field Constructor
-* @param name name of the field
-* @param type datatype of field, specified in strings.
-*/
-public Field(String name, String type);
-</code></pre>
-<pre><code>/**
-* Field constructor
-* @param name name of the field
-* @param type datatype of the field of class DataType
-*/
-public Field(String name, DataType type);  
-</code></pre>
-<h3>
-<a id="class-orgapachecarbondatasdkfileschema" class="anchor" href="#class-orgapachecarbondatasdkfileschema" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.Schema</h3>
-<pre><code>/**
-* construct a schema with fields
-* @param fields
-*/
-public Schema(Field[] fields);
-</code></pre>
-<pre><code>/**
-* Create a Schema using JSON string, for example:
-* [
-*   {"name":"string"},
-*   {"age":"int"}
-* ] 
-* @param json specified as string
-* @return Schema
-*/
-public static Schema parseJson(String json);
-</code></pre>
-<h3>
-<a id="class-orgapachecarbondatasdkfileavrocarbonwriter" class="anchor" href="#class-orgapachecarbondatasdkfileavrocarbonwriter" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Class org.apache.carbondata.sdk.file.AvroCarbonWriter</h3>
-<pre><code>/**
-* converts avro schema to carbon schema, required by carbonWriter
-*
-* @param avroSchemaString json formatted avro schema as string
-* @return carbon sdk schema
-*/
-public static org.apache.carbondata.sdk.file.Schema getCarbonSchemaFromAvroSchema(String avroSchemaString);
-</code></pre>
-</div>
-</div>
-</div>
-</div>
-<div class="doc-footer">
-    <a href="#top" class="scroll-top">Top</a>
-</div>
-</div>
-</section>
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/security.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/security.html b/src/main/webapp/security.html
index 2aa4b8e..9168fa3 100644
--- a/src/main/webapp/security.html
+++ b/src/main/webapp/security.html
@@ -205,4 +205,4 @@
 <script src="js/custom.js"></script>
 <script src="js/mdNavigation.js" type="text/javascript"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/segment-management-on-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/segment-management-on-carbondata.html b/src/main/webapp/segment-management-on-carbondata.html
index 528ee9d..1e6f61d 100644
--- a/src/main/webapp/segment-management-on-carbondata.html
+++ b/src/main/webapp/segment-management-on-carbondata.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -307,8 +307,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -323,4 +322,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/streaming-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/streaming-guide.html b/src/main/webapp/streaming-guide.html
index c6d8391..86f0385 100644
--- a/src/main/webapp/streaming-guide.html
+++ b/src/main/webapp/streaming-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -212,6 +212,37 @@
                                     <div>
 <h1>
 <a id="carbondata-streaming-ingestion" class="anchor" href="#carbondata-streaming-ingestion" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Streaming Ingestion</h1>
+<ul>
+<li>
+<a href="#quick-example">Streaming Table Management</a>
+<ul>
+<li><a href="#create-table-with-streaming-property">Create table with streaming property</a></li>
+<li><a href="#alter-streaming-property">Alter streaming property</a></li>
+<li><a href="#acquire-streaming-lock">Acquire streaming lock</a></li>
+<li><a href="#create-streaming-segment">Create streaming segment</a></li>
+<li><a href="#change-segment-status">Change Stream segment status</a></li>
+<li><a href="#handoff-streaming-finish-segment-to-columnar-segment">Handoff "streaming finish" segment to columnar segment</a></li>
+<li><a href="#auto-handoff-streaming-segment">Auto handoff streaming segment</a></li>
+<li><a href="#stream-data-parser">Stream data parser</a></li>
+<li><a href="#close-streaming-table">Close streaming table</a></li>
+<li><a href="#constraint">Constraints</a></li>
+</ul>
+</li>
+<li>
+<a href="#streamsql">StreamSQL</a>
+<ul>
+<li><a href="#streaming-table">Defining Streaming Table</a></li>
+<li>
+<a href="#streaming-job-management">Streaming Job Management</a>
+<ul>
+<li><a href="#start-stream">START STREAM</a></li>
+<li><a href="#stop-stream">STOP STREAM</a></li>
+<li><a href="#show-streams">SHOW STREAMS</a></li>
+</ul>
+</li>
+</ul>
+</li>
+</ul>
 <h2>
 <a id="quick-example" class="anchor" href="#quick-example" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Quick example</h2>
 <p>Download and unzip spark-2.2.0-bin-hadoop2.7.tgz, and export $SPARK_HOME</p>
@@ -255,7 +286,7 @@
 <span class="pl-s">      | col1 INT,</span>
 <span class="pl-s">      | col2 STRING</span>
 <span class="pl-s">      | )</span>
-<span class="pl-s">      | STORED BY 'carbondata'</span>
+<span class="pl-s">      | STORED AS carbondata</span>
 <span class="pl-s">      | TBLPROPERTIES('streaming'='true')<span class="pl-pds">"""</span></span>.stripMargin)
 
  <span class="pl-k">val</span> <span class="pl-smi">carbonTable</span> <span class="pl-k">=</span> <span class="pl-en">CarbonEnv</span>.getCarbonTable(<span class="pl-en">Some</span>(<span class="pl-s"><span class="pl-pds">"</span>default<span class="pl-pds">"</span></span>), <span class="pl-s"><span class="pl-pds">"</span>carbon_table<span class="pl-pds">"</span></span>)(spark)
@@ -300,7 +331,7 @@ streaming table using following DDL.</p>
   col1 <span class="pl-k">INT</span>,
   col2 STRING
  )
- STORED BY <span class="pl-s"><span class="pl-pds">'</span>carbondata<span class="pl-pds">'</span></span>
+ STORED <span class="pl-k">AS</span> carbondata
  TBLPROPERTIES(<span class="pl-s"><span class="pl-pds">'</span>streaming<span class="pl-pds">'</span></span><span class="pl-k">=</span><span class="pl-s"><span class="pl-pds">'</span>true<span class="pl-pds">'</span></span>)</pre></div>
 <table>
 <thead>
@@ -483,6 +514,145 @@ streaming table using following DDL.</p>
 <li>block delete "streaming" segment while the streaming ingestion is running.</li>
 <li>block drop the streaming table while the streaming ingestion is running.</li>
 </ol>
+<h2>
+<a id="streamsql" class="anchor" href="#streamsql" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>StreamSQL</h2>
+<h3>
+<a id="streaming-table" class="anchor" href="#streaming-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Streaming Table</h3>
+<p><strong>Example</strong></p>
+<p>Following example shows how to start a streaming ingest job</p>
+<pre><code>    sql(
+      s"""
+         |CREATE TABLE source(
+         | id INT,
+         | name STRING,
+         | city STRING,
+         | salary FLOAT,
+         | tax DECIMAL(8,2),
+         | percent double,
+         | birthday DATE,
+         | register TIMESTAMP,
+         | updated TIMESTAMP
+         |)
+         |STORED BY carbondata
+         |TBLPROPERTIES (
+         | 'format'='csv',
+         | 'path'='$csvDataDir'
+         |)
+      """.stripMargin)
+
+    sql(
+      s"""
+         |CREATE TABLE sink(
+         | id INT,
+         | name STRING,
+         | city STRING,
+         | salary FLOAT,
+         | tax DECIMAL(8,2),
+         | percent double,
+         | birthday DATE,
+         | register TIMESTAMP,
+         | updated TIMESTAMP
+         |)
+         |STORED BY carbondata
+         |TBLPROPERTIES (
+         |  'streaming'='true'
+         |)
+      """.stripMargin)
+
+    sql(
+      """
+        |START STREAM job123 ON TABLE sink
+        |STMPROPERTIES(
+        |  'trigger'='ProcessingTime',
+        |  'interval'='1 seconds')
+        |AS
+        |  SELECT *
+        |  FROM source
+        |  WHERE id % 2 = 1
+      """.stripMargin)
+
+    sql("STOP STREAM job123")
+
+    sql("SHOW STREAMS [ON TABLE tableName]")
+</code></pre>
+<p>In above example, two table is created: source and sink. The <code>source</code> table's format is <code>csv</code> and <code>sink</code> table format is <code>carbon</code>. Then a streaming job is created to stream data from source table to sink table.</p>
+<p>These two tables are normal carbon table, they can be queried independently.</p>
+<h3>
+<a id="streaming-job-management" class="anchor" href="#streaming-job-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Streaming Job Management</h3>
+<p>As above example shown:</p>
+<ul>
+<li>
+<code>START STREAM jobName ON TABLE tableName</code> is used to start a streaming ingest job.</li>
+<li>
+<code>STOP STREAM jobName</code> is used to stop a streaming job by its name</li>
+<li>
+<code>SHOW STREAMS [ON TABLE tableName]</code> is used to print streaming job information</li>
+</ul>
+<h5>
+<a id="start-stream" class="anchor" href="#start-stream" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>START STREAM</h5>
+<p>When this is issued, carbon will start a structured streaming job to do the streaming ingestion. Before launching the job, system will validate:</p>
+<ul>
+<li>
+<p>The format of table specified in CTAS FROM clause must be one of: csv, json, text, parquet, kafka, socket.  These are formats supported by spark 2.2.0 structured streaming</p>
+</li>
+<li>
+<p>User should pass the options of the streaming source table in its TBLPROPERTIES when creating it. StreamSQL will pass them transparently to spark when creating the streaming job. For example:</p>
+<div class="highlight highlight-source-sql"><pre><span class="pl-k">CREATE</span> <span class="pl-k">TABLE</span> <span class="pl-en">source</span>(
+  name STRING,
+  age <span class="pl-k">INT</span>
+)
+STORED BY carbondata
+TBLPROPERTIES(
+  <span class="pl-s"><span class="pl-pds">'</span>format<span class="pl-pds">'</span></span><span class="pl-k">=</span><span class="pl-s"><span class="pl-pds">'</span>socket<span class="pl-pds">'</span></span>,
+  <span class="pl-s"><span class="pl-pds">'</span>host<span class="pl-pds">'</span></span><span class="pl-k">=</span><span class="pl-s"><span class="pl-pds">'</span>localhost<span class="pl-pds">'</span></span>,
+  <span class="pl-s"><span class="pl-pds">'</span>port<span class="pl-pds">'</span></span><span class="pl-k">=</span><span class="pl-s"><span class="pl-pds">'</span>8888<span class="pl-pds">'</span></span>
+)</pre></div>
+<p>will translate to</p>
+<div class="highlight highlight-source-scala"><pre>spark.readStream
+	 .schema(tableSchema)
+	 .format(<span class="pl-s"><span class="pl-pds">"</span>socket<span class="pl-pds">"</span></span>)
+	 .option(<span class="pl-s"><span class="pl-pds">"</span>host<span class="pl-pds">"</span></span>, <span class="pl-s"><span class="pl-pds">"</span>localhost<span class="pl-pds">"</span></span>)
+	 .option(<span class="pl-s"><span class="pl-pds">"</span>port<span class="pl-pds">"</span></span>, <span class="pl-s"><span class="pl-pds">"</span>8888<span class="pl-pds">"</span></span>)</pre></div>
+</li>
+<li>
+<p>The sink table should have a TBLPROPERTY <code>'streaming'</code> equal to <code>true</code>, indicating it is a streaming table.</p>
+</li>
+<li>
+<p>In the given STMPROPERTIES, user must specify <code>'trigger'</code>, its value must be <code>ProcessingTime</code> (In future, other value will be supported). User should also specify interval value for the streaming job.</p>
+</li>
+<li>
+<p>If the schema specifid in sink table is different from CTAS, the streaming job will fail</p>
+</li>
+</ul>
+<h5>
+<a id="stop-stream" class="anchor" href="#stop-stream" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>STOP STREAM</h5>
+<p>When this is issued, the streaming job will be stopped immediately. It will fail if the jobName specified is not exist.</p>
+<h5>
+<a id="show-streams" class="anchor" href="#show-streams" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SHOW STREAMS</h5>
+<p><code>SHOW STREAMS ON TABLE tableName</code> command will print the streaming job information as following</p>
+<table>
+<thead>
+<tr>
+<th>Job name</th>
+<th>status</th>
+<th>Source</th>
+<th>Sink</th>
+<th>start time</th>
+<th>time elapsed</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>job123</td>
+<td>Started</td>
+<td>device</td>
+<td>fact</td>
+<td>2018-02-03 14:32:42</td>
+<td>10d2h32m</td>
+</tr>
+</tbody>
+</table>
+<p><code>SHOW STREAMS</code> command will show all stream jobs in the system.</p>
 <script>
 $(function() {
   // Show selected style on nav item
@@ -493,8 +663,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -509,4 +678,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/supported-data-types-in-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/supported-data-types-in-carbondata.html b/src/main/webapp/supported-data-types-in-carbondata.html
index e0ad647..f346052 100644
--- a/src/main/webapp/supported-data-types-in-carbondata.html
+++ b/src/main/webapp/supported-data-types-in-carbondata.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -269,8 +269,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -285,4 +284,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/timeseries-datamap-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/timeseries-datamap-guide.html b/src/main/webapp/timeseries-datamap-guide.html
index 497b02f..73a4580 100644
--- a/src/main/webapp/timeseries-datamap-guide.html
+++ b/src/main/webapp/timeseries-datamap-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -213,9 +213,9 @@
 <h1>
 <a id="carbondata-timeseries-datamap" class="anchor" href="#carbondata-timeseries-datamap" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Timeseries DataMap</h1>
 <ul>
-<li><a href="#timeseries-datamap-intoduction">Timeseries DataMap Introduction</a></li>
-<li><a href="#compacting-pre-aggregate-tables">Compaction</a></li>
-<li><a href="#data-management-with-pre-aggregate-tables">Data Management</a></li>
+<li><a href="#timeseries-datamap-introduction-alpha-feature">Timeseries DataMap Introduction</a></li>
+<li><a href="#compacting-timeseries-datamp">Compaction</a></li>
+<li><a href="#data-management-on-timeseries-datamap">Data Management</a></li>
 </ul>
 <h2>
 <a id="timeseries-datamap-introduction-alpha-feature" class="anchor" href="#timeseries-datamap-introduction-alpha-feature" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Timeseries DataMap Introduction (Alpha Feature)</h2>
@@ -343,8 +343,7 @@ $(function() {
     $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -359,4 +358,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file


[16/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/dml-of-carbondata.html
----------------------------------------------------------------------
diff --git a/content/dml-of-carbondata.html b/content/dml-of-carbondata.html
new file mode 100644
index 0000000..2048c9e
--- /dev/null
+++ b/content/dml-of-carbondata.html
@@ -0,0 +1,588 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="carbondata-data-manipulation-language" class="anchor" href="#carbondata-data-manipulation-language" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Data Manipulation Language</h1>
+<p>CarbonData DML statements are documented here,which includes:</p>
+<ul>
+<li><a href="#load-data">LOAD DATA</a></li>
+<li><a href="#insert-data-into-carbondata-table">INSERT DATA</a></li>
+<li><a href="#load-data-using-static-partition">Load Data Using Static Partition</a></li>
+<li><a href="#load-data-using-dynamic-partition">Load Data Using Dynamic Partition</a></li>
+<li><a href="#update-and-delete">UPDATE AND DELETE</a></li>
+<li><a href="#compaction">COMPACTION</a></li>
+<li><a href="./segment-management-on-carbondata.html">SEGMENT MANAGEMENT</a></li>
+</ul>
+<h2>
+<a id="load-data" class="anchor" href="#load-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LOAD DATA</h2>
+<h3>
+<a id="load-files-to-carbondata-table" class="anchor" href="#load-files-to-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>LOAD FILES TO CARBONDATA TABLE</h3>
+<p>This command is used to load csv files to carbondata, OPTIONS are not mandatory for data loading process.</p>
+<pre><code>LOAD DATA [LOCAL] INPATH 'folder_path' 
+INTO TABLE [db_name.]table_name 
+OPTIONS(property_name=property_value, ...)
+</code></pre>
+<p><strong>Supported Properties:</strong> <a href="#delimiter">DELIMITER</a>, <a href="#quotechar">QUOTECHAR</a>, <a href="#commentchar">COMMENTCHAR</a>, <a href="#header">HEADER</a>, <a href="#fileheader">FILEHEADER</a>, <a href="#multiline">MULTILINE</a>, <a href="#escapechar">ESCAPECHAR</a>, <a href="#skip_empty_line">SKIP_EMPTY_LINE</a>, <a href="#complex_delimiter_level_1">COMPLEX_DELIMITER_LEVEL_1</a>, <a href="#complex_delimiter_level_2">COMPLEX_DELIMITER_LEVEL_2</a>, <a href="#all_dictionary_path">ALL_DICTIONARY_PATH</a>, <a href="#columndict">COLUMNDICT</a>, <a href="#dateformat">DATEFORMAT</a>,<a href="#timestampformat"> TIMESTAMPFORMAT</a>, <a href="#sort-column-bounds">SORT_COLUMN_BOUNDS</a>, <a href="#single_pass">SINGLE_PASS</a>, <a href="#bad-records-handling">BAD_RECORDS_LOGGER_ENABLE</a>, <a href="#bad-records-handling">BAD_RECORD_PATH</a>, <a href="#bad-records-handling">BAD_RECORDS_ACTION</a>, <a href="#bad-records-handling">IS_EMPTY_DATA_BAD_RECORD</a>, <a href="
 #global_sort_partitions">GLOBAL_SORT_PARTITIONS</a></p>
+<p>You can use the following options to load data:</p>
+<ul>
+<li>
+<h5>
+<a id="delimiter" class="anchor" href="#delimiter" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELIMITER:</h5>
+<p>Delimiters can be provided in the load command.</p>
+<pre><code>OPTIONS('DELIMITER'=',')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="quotechar" class="anchor" href="#quotechar" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>QUOTECHAR:</h5>
+<p>Quote Characters can be provided in the load command.</p>
+<pre><code>OPTIONS('QUOTECHAR'='"')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="commentchar" class="anchor" href="#commentchar" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COMMENTCHAR:</h5>
+<p>Comment Characters can be provided in the load command if user want to comment lines.</p>
+<pre><code>OPTIONS('COMMENTCHAR'='#')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="header" class="anchor" href="#header" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>HEADER:</h5>
+<p>When you load the CSV file without the file header and the file header is the same with the table schema, then add 'HEADER'='false' to load data SQL as user need not provide the file header. By default the value is 'true'.
+false: CSV file is without file header.
+true: CSV file is with file header.</p>
+<pre><code>OPTIONS('HEADER'='false') 
+</code></pre>
+<p><strong>NOTE:</strong> If the HEADER option exist and is set to 'true', then the FILEHEADER option is not required.</p>
+</li>
+<li>
+<h5>
+<a id="fileheader" class="anchor" href="#fileheader" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>FILEHEADER:</h5>
+<p>Headers can be provided in the LOAD DATA command if headers are missing in the source files.</p>
+<pre><code>OPTIONS('FILEHEADER'='column1,column2') 
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="multiline" class="anchor" href="#multiline" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>MULTILINE:</h5>
+<p>CSV with new line character in quotes.</p>
+<pre><code>OPTIONS('MULTILINE'='true') 
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="escapechar" class="anchor" href="#escapechar" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>ESCAPECHAR:</h5>
+<p>Escape char can be provided if user want strict validation of escape character in CSV files.</p>
+<pre><code>OPTIONS('ESCAPECHAR'='\') 
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="skip_empty_line" class="anchor" href="#skip_empty_line" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SKIP_EMPTY_LINE:</h5>
+<p>This option will ignore the empty line in the CSV file during the data load.</p>
+<pre><code>OPTIONS('SKIP_EMPTY_LINE'='TRUE/FALSE') 
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="complex_delimiter_level_1" class="anchor" href="#complex_delimiter_level_1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COMPLEX_DELIMITER_LEVEL_1:</h5>
+<p>Split the complex type data column in a row (eg., a$b$c --&gt; Array = {a,b,c}).</p>
+<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_1'='$') 
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="complex_delimiter_level_2" class="anchor" href="#complex_delimiter_level_2" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COMPLEX_DELIMITER_LEVEL_2:</h5>
+<p>Split the complex type nested data column in a row. Applies level_1 delimiter &amp; applies level_2 based on complex data type (eg., a:b$c:d --&gt; Array&gt; = {{a,b},{c,d}}).</p>
+<pre><code>OPTIONS('COMPLEX_DELIMITER_LEVEL_2'=':')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="all_dictionary_path" class="anchor" href="#all_dictionary_path" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>ALL_DICTIONARY_PATH:</h5>
+<p>All dictionary files path.</p>
+<pre><code>OPTIONS('ALL_DICTIONARY_PATH'='/opt/alldictionary/data.dictionary')
+</code></pre>
+</li>
+<li>
+<h5>
+<a id="columndict" class="anchor" href="#columndict" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COLUMNDICT:</h5>
+<p>Dictionary file path for specified column.</p>
+<pre><code>OPTIONS('COLUMNDICT'='column1:dictionaryFilePath1,column2:dictionaryFilePath2')
+</code></pre>
+<p><strong>NOTE:</strong> ALL_DICTIONARY_PATH and COLUMNDICT can't be used together.</p>
+</li>
+<li>
+<h5>
+<a id="dateformattimestampformat" class="anchor" href="#dateformattimestampformat" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DATEFORMAT/TIMESTAMPFORMAT:</h5>
+<p>Date and Timestamp format for specified column.</p>
+<pre><code>OPTIONS('DATEFORMAT' = 'yyyy-MM-dd','TIMESTAMPFORMAT'='yyyy-MM-dd HH:mm:ss')
+</code></pre>
+<p><strong>NOTE:</strong> Date formats are specified by date pattern strings. The date pattern letters in CarbonData are same as in JAVA. Refer to <a href="http://docs.oracle.com/javase/7/docs/api/java/text/SimpleDateFormat.html" target=_blank rel="nofollow">SimpleDateFormat</a>.</p>
+</li>
+<li>
+<h5>
+<a id="sort-column-bounds" class="anchor" href="#sort-column-bounds" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SORT COLUMN BOUNDS:</h5>
+<p>Range bounds for sort columns.</p>
+<p>Suppose the table is created with 'SORT_COLUMNS'='name,id' and the range for name is aaa<del>zzz, the value range for id is 0</del>1000. Then during data loading, we can specify the following option to enhance data loading performance.</p>
+<pre><code>OPTIONS('SORT_COLUMN_BOUNDS'='f,250;l,500;r,750')
+</code></pre>
+<p>Each bound is separated by ';' and each field value in bound is separated by ','. In the example above, we provide 3 bounds to distribute records to 4 partitions. The values 'f','l','r' can evenly distribute the records. Inside carbondata, for a record we compare the value of sort columns with that of the bounds and decide which partition the record will be forwarded to.</p>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>SORT_COLUMN_BOUNDS will be used only when the SORT_SCOPE is 'local_sort'.</li>
+<li>Carbondata will use these bounds as ranges to process data concurrently during the final sort percedure. The records will be sorted and written out inside each partition. Since the partition is sorted, all records will be sorted.</li>
+<li>Since the actual order and literal order of the dictionary column are not necessarily the same, we do not recommend you to use this feature if the first sort column is 'dictionary_include'.</li>
+<li>The option works better if your CPU usage during loading is low. If your system is already CPU tense, better not to use this option. Besides, it depends on the user to specify the bounds. If user does not know the exactly bounds to make the data distributed evenly among the bounds, loading performance will still be better than before or at least the same as before.</li>
+<li>Users can find more information about this option in the description of PR1953.</li>
+</ul>
+</li>
+<li>
+<h5>
+<a id="single_pass" class="anchor" href="#single_pass" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SINGLE_PASS:</h5>
+<p>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.</p>
+</li>
+</ul>
+<p>This option specifies whether to use single pass for loading data or not. By default this option is set to FALSE.</p>
+<pre><code> OPTIONS('SINGLE_PASS'='TRUE')
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>If this option is set to TRUE then data loading will take less time.</li>
+<li>If this option is set to some invalid value other than TRUE or FALSE then it uses the default value.</li>
+</ul>
+<p>Example:</p>
+<pre><code>LOAD DATA local inpath '/opt/rawdata/data.csv' INTO table carbontable
+options('DELIMITER'=',', 'QUOTECHAR'='"','COMMENTCHAR'='#',
+'HEADER'='false',
+'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')
+</code></pre>
+<ul>
+<li>
+<h5>
+<a id="bad-records-handling" class="anchor" href="#bad-records-handling" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>BAD RECORDS HANDLING:</h5>
+<p>Methods of handling bad records are as follows:</p>
+<ul>
+<li>Load all of the data before dealing with the errors.</li>
+<li>Clean or delete bad records before loading data or stop the loading when bad records are found.</li>
+</ul>
+<pre><code>OPTIONS('BAD_RECORDS_LOGGER_ENABLE'='true', 'BAD_RECORD_PATH'='hdfs://hacluster/tmp/carbon', 'BAD_RECORDS_ACTION'='REDIRECT', 'IS_EMPTY_DATA_BAD_RECORD'='false')
+</code></pre>
+</li>
+</ul>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>BAD_RECORDS_ACTION property can have four type of actions for bad records FORCE, REDIRECT, IGNORE and FAIL.</li>
+<li>FAIL option is its Default value. If the FAIL option is used, then data loading fails if any bad records are found.</li>
+<li>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.</li>
+<li>If the FORCE option is used, then it auto-converts the data by storing the bad records as NULL before Loading data.</li>
+<li>If the IGNORE option is used, then bad records are neither loaded nor written to the separate CSV file.</li>
+<li>In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails.</li>
+<li>The default maximum number of characters per column is 32000. If there are more than 32000 characters in a column, please refer to <em>String longer than 32000 characters</em> section.</li>
+<li>Since Bad Records Path can be specified in create, load and carbon properties.
+Therefore, value specified in load will have the highest priority, and value specified in carbon properties will have the least priority.</li>
+</ul>
+<p><strong>Bad Records Path:</strong>
+This property is used to specify the location where bad records would be written.</p>
+<pre><code>TBLPROPERTIES('BAD_RECORDS_PATH'='/opt/badrecords'')
+</code></pre>
+<p>Example:</p>
+<pre><code>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')
+</code></pre>
+<ul>
+<li>
+<h5>
+<a id="global_sort_partitions" class="anchor" href="#global_sort_partitions" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>GLOBAL_SORT_PARTITIONS:</h5>
+<p>If the SORT_SCOPE is defined as GLOBAL_SORT, then user can specify the number of partitions to use while shuffling data for sort using GLOBAL_SORT_PARTITIONS. If it is not configured, or configured less than 1, then it uses the number of map task as reduce task. It is recommended that each reduce task deal with 512MB-1GB data.</p>
+</li>
+</ul>
+<pre><code>OPTIONS('GLOBAL_SORT_PARTITIONS'='2')
+</code></pre>
+<p>NOTE:</p>
+<ul>
+<li>GLOBAL_SORT_PARTITIONS should be Integer type, the range is [1,Integer.MaxValue].</li>
+<li>It is only used when the SORT_SCOPE is GLOBAL_SORT.</li>
+</ul>
+<h3>
+<a id="insert-data-into-carbondata-table" class="anchor" href="#insert-data-into-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>INSERT DATA INTO CARBONDATA TABLE</h3>
+<p>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.</p>
+<pre><code>INSERT INTO TABLE &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p>You can also omit the <code>table</code> keyword and write your query as:</p>
+<pre><code>INSERT INTO &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p>Overwrite insert data:</p>
+<pre><code>INSERT OVERWRITE TABLE &lt;CARBONDATA TABLE&gt; SELECT * FROM sourceTableName 
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p><strong>NOTE:</strong></p>
+<ul>
+<li>The source table and the CarbonData table must have the same table schema.</li>
+<li>The data type of source and destination table columns should be same</li>
+<li>INSERT INTO command does not support partial success if bad records are found, it will fail.</li>
+<li>Data cannot be loaded or updated in source table while insert from source table to target table is in progress.</li>
+</ul>
+<p>Examples</p>
+<pre><code>INSERT INTO table1 SELECT item1, sum(item2 + 1000) as result FROM table2 group by item1
+</code></pre>
+<pre><code>INSERT INTO table1 SELECT item1, item2, item3 FROM table2 where item2='xyz'
+</code></pre>
+<pre><code>INSERT OVERWRITE TABLE table1 SELECT * FROM TABLE2
+</code></pre>
+<h3>
+<a id="load-data-using-static-partition" class="anchor" href="#load-data-using-static-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Load Data Using Static Partition</h3>
+<p>This command allows you to load data using static partition.</p>
+<pre><code>LOAD DATA [LOCAL] INPATH 'folder_path' 
+INTO TABLE [db_name.]table_name PARTITION (partition_spec) 
+OPTIONS(property_name=property_value, ...)    
+INSERT INTO INTO TABLE [db_name.]table_name PARTITION (partition_spec) &lt;SELECT STATEMENT&gt;
+</code></pre>
+<p>Example:</p>
+<pre><code>LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.csv'
+INTO TABLE locationTable
+PARTITION (country = 'US', state = 'CA')  
+INSERT INTO TABLE locationTable
+PARTITION (country = 'US', state = 'AL')
+SELECT &lt;columns list excluding partition columns&gt; FROM another_user
+</code></pre>
+<h3>
+<a id="load-data-using-dynamic-partition" class="anchor" href="#load-data-using-dynamic-partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Load Data Using Dynamic Partition</h3>
+<p>This command allows you to load data using dynamic partition. If partition spec is not specified, then the partition is considered as dynamic.</p>
+<p>Example:</p>
+<pre><code>LOAD DATA LOCAL INPATH '${env:HOME}/staticinput.csv'
+INTO TABLE locationTable          
+INSERT INTO TABLE locationTable
+SELECT &lt;columns list excluding partition columns&gt; FROM another_user
+</code></pre>
+<h2>
+<a id="update-and-delete" class="anchor" href="#update-and-delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>UPDATE AND DELETE</h2>
+<h3>
+<a id="update" class="anchor" href="#update" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>UPDATE</h3>
+<p>This command will allow to update the CarbonData table based on the column expression and optional filter conditions.</p>
+<pre><code>UPDATE &lt;table_name&gt; 
+SET (column_name1, column_name2, ... column_name n) = (column1_expression , column2_expression, ... column n_expression )
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p>alternatively the following command can also be used for updating the CarbonData Table :</p>
+<pre><code>UPDATE &lt;table_name&gt;
+SET (column_name1, column_name2) =(select sourceColumn1, sourceColumn2 from sourceTable [ WHERE { &lt;filter_condition&gt; } ] )
+[ WHERE { &lt;filter_condition&gt; } ]
+</code></pre>
+<p><strong>NOTE:</strong> The update command fails if multiple input rows in source table are matched with single row in destination table.</p>
+<p>Examples:</p>
+<pre><code>UPDATE t3 SET (t3_salary) = (t3_salary + 9) WHERE t3_name = 'aaa1'
+</code></pre>
+<pre><code>UPDATE t3 SET (t3_date, t3_country) = ('2017-11-18', 'india') WHERE t3_salary &lt; 15003
+</code></pre>
+<pre><code>UPDATE t3 SET (t3_country, t3_name) = (SELECT t5_country, t5_name FROM t5 WHERE t5_id = 5) WHERE t3_id &lt; 5
+</code></pre>
+<pre><code>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 &lt; 5
+</code></pre>
+<pre><code>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 &gt;6
+</code></pre>
+<p>NOTE: Update Complex datatype columns is not supported.</p>
+<h3>
+<a id="delete" class="anchor" href="#delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DELETE</h3>
+<p>This command allows us to delete records from CarbonData table.</p>
+<pre><code>DELETE FROM table_name [WHERE expression]
+</code></pre>
+<p>Examples:</p>
+<pre><code>DELETE FROM carbontable WHERE column1  = 'china'
+</code></pre>
+<pre><code>DELETE FROM carbontable WHERE column1 IN ('china', 'USA')
+</code></pre>
+<pre><code>DELETE FROM carbontable WHERE column1 IN (SELECT column11 FROM sourceTable2)
+</code></pre>
+<pre><code>DELETE FROM carbontable WHERE column1 IN (SELECT column11 FROM sourceTable2 WHERE column1 = 'USA')
+</code></pre>
+<h2>
+<a id="compaction" class="anchor" href="#compaction" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>COMPACTION</h2>
+<p>Compaction improves the query performance significantly.</p>
+<p>There are several types of compaction.</p>
+<pre><code>ALTER TABLE [db_name.]table_name COMPACT 'MINOR/MAJOR/CUSTOM'
+</code></pre>
+<ul>
+<li><strong>Minor Compaction</strong></li>
+</ul>
+<p>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:</p>
+<ul>
+<li>Level 1: Merging of the segments which are not yet compacted.</li>
+<li>Level 2: Merging of the compacted segments again to form a larger segment.</li>
+</ul>
+<pre><code>ALTER TABLE table_name COMPACT 'MINOR'
+</code></pre>
+<ul>
+<li><strong>Major Compaction</strong></li>
+</ul>
+<p>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.
+Configure the property carbon.major.compaction.size with appropriate value in MB.</p>
+<p>This command merges the specified number of segments into one segment:</p>
+<pre><code>ALTER TABLE table_name COMPACT 'MAJOR'
+</code></pre>
+<ul>
+<li><strong>Custom Compaction</strong></li>
+</ul>
+<p>In Custom compaction, user can directly specify segment ids to be merged into one large segment.
+All specified segment ids should exist and be valid, otherwise compaction will fail.
+Custom compaction is usually done during the off-peak time.</p>
+<pre><code>ALTER TABLE table_name COMPACT 'CUSTOM' WHERE SEGMENT.ID IN (2,3,4)
+</code></pre>
+<p>NOTE: Compaction is unsupported for table containing Complex columns.</p>
+<ul>
+<li><strong>CLEAN SEGMENTS AFTER Compaction</strong></li>
+</ul>
+<p>Clean the segments which are compacted:</p>
+<pre><code>CLEAN FILES FOR TABLE carbon_table
+</code></pre>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/dml-operation-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/dml-operation-on-carbondata.html b/content/dml-operation-on-carbondata.html
index b6a5642..655a61c 100644
--- a/content/dml-operation-on-carbondata.html
+++ b/content/dml-operation-on-carbondata.html
@@ -713,4 +713,4 @@ column1 = 'USA');
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/documentation.html
----------------------------------------------------------------------
diff --git a/content/documentation.html b/content/documentation.html
new file mode 100644
index 0000000..634da94
--- /dev/null
+++ b/content/documentation.html
@@ -0,0 +1,256 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
+                                   target="_blank">Apache CarbonData 1.4.1</a></li>
+							<li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div>
+<h1>
+<a id="apache-carbondata-documentation" class="anchor" href="#apache-carbondata-documentation" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Apache CarbonData Documentation</h1>
+<p>Apache CarbonData is a new big data file format for faster interactive query using advanced columnar storage, index, compression and encoding techniques to improve computing efficiency, which helps in speeding up queries by an order of magnitude faster over PetaBytes of data.</p>
+<h2>
+<a id="getting-started" class="anchor" href="#getting-started" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Getting Started</h2>
+<p><strong>File Format Concepts:</strong> Start with the basics of understanding the <a href="./file-structure-of-carbondata.html#carbondata-file-structure">CarbonData file format</a> and its storage structure.This will help to understand other parts of the documentation, incuding deployment, programming and usage guides.</p>
+<p><strong>Quick Start:</strong> <a href="./quick-start-guide.html#installing-and-configuring-carbondata-to-run-locally-with-spark-shell">Run an example program</a> on your local machine or <a href="https://github.com/apache/carbondata/tree/master/examples/spark2/src/main/scala/org/apache/carbondata/examples" target=_blank>study some examples</a>.</p>
+<p><strong>CarbonData SQL Language Reference:</strong> CarbonData extends the Spark SQL language and adds several <a href="./ddl-of-carbondata.html">DDL</a> and <a href="./dml-of-carbondata.html">DML</a> statements to support operations on it.Refer to the <a href="./language-manual.html">Reference Manual</a> to understand the supported features and functions.</p>
+<p><strong>Programming Guides:</strong> You can read our guides about <a href="./sdk-guide.html">APIs supported</a> to learn how to integrate CarbonData with your applications.</p>
+<h2>
+<a id="deployment" class="anchor" href="#deployment" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Deployment</h2>
+<p>CarbonData can be integrated with popular Execution engines like <a href="./quick-start-guide.html#spark">Spark</a> and <a href="./quick-start-guide.html#presto">Presto</a>.Refer to the <a href="./quick-start-guide.html##deployment-modes">Installation and Configuration</a> section to understand all modes of Integrating CarbonData.</p>
+<h2>
+<a id="contributing-to-carbondata" class="anchor" href="#contributing-to-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Contributing to CarbonData</h2>
+<p>The Apache CarbonData community welcomes all kinds of contributions from anyone with a passion for
+faster data format.Contributing to CarbonData doesn?t just mean writing code. Helping new users on the mailing list, testing releases, and improving documentation are also welcome.Please follow the <a href="./how-to-contribute-to-apache-carbondata.html">Contributing to CarbonData guidelines</a> before proposing a design or code change.</p>
+<p><strong>Compiling CarbonData:</strong> This <a href="https://github.com/apache/carbondata/tree/master/build" target=_blank>guide</a> will help you to compile and generate the jars for test.</p>
+<h2>
+<a id="external-resources" class="anchor" href="#external-resources" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>External Resources</h2>
+<p><strong>Wiki:</strong> You can read the <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/CarbonData+Home" rel="nofollow">Apache CarbonData wiki</a> page for upcoming release plan, blogs and training materials.</p>
+<p><strong>Summit:</strong> Presentations from past summits and conferences can be found <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609" rel="nofollow">here</a>.</p>
+<p><strong>Blogs:</strong> Blogs by external users can be found <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=67635497" rel="nofollow">here</a>.</p>
+<p><strong>Performance reports:</strong> TPC-H performance reports can be found <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Performance+-+TPCH+Report+of+CarbonData+%281.2+version%29+and+Parquet+on+Spark+Execution+Engine" rel="nofollow">here</a>.</p>
+<p><strong>Trainings:</strong> Training records on design and code flows can be found <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/CarbonData+Training+Materials" rel="nofollow">here</a>.</p>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__intro').addClass('selected'); });
+</script>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/errorpage.html
----------------------------------------------------------------------
diff --git a/content/errorpage.html b/content/errorpage.html
index 090dce5..35cc01a 100644
--- a/content/errorpage.html
+++ b/content/errorpage.html
@@ -89,4 +89,4 @@
 
 
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/faq.html
----------------------------------------------------------------------
diff --git a/content/faq.html b/content/faq.html
index cf84d34..52112b7 100644
--- a/content/faq.html
+++ b/content/faq.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -174,6 +225,22 @@
 <li><a href="#Why-all-executors-are-showing-success-in-Spark-UI-even-after-Dataload-command-failed-at-driver-side">Why all executors are showing success in Spark UI even after Dataload command failed at Driver side?</a></li>
 <li><a href="#Why-different-time-zone-result-for-select-query-output-when-query-SDK-writer-output">Why different time zone result for select query output when query SDK writer output?</a></li>
 </ul>
+<h1>
+<a id="troubleshooting" class="anchor" href="#troubleshooting" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>TroubleShooting</h1>
+<ul>
+<li><a href="#Getting-tablestatus.lock-issues-When-loading-data">Getting tablestatus.lock issues When loading data</a></li>
+<li><a href="#failed-to-load-thrift-libraries">Failed to load thrift libraries</a></li>
+<li><a href="#failed-to-launch-the-spark-shell">Failed to launch the Spark Shell</a></li>
+<li><a href="#failed-to-execute-load-query-on-cluster">Failed to execute load query on cluster</a></li>
+<li><a href="#failed-to-execute-insert-query-on-cluster">Failed to execute insert query on cluster</a></li>
+<li><a href="#failed-to-connect-to-hiveuser-with-thrift">Failed to connect to hiveuser with thrift</a></li>
+<li><a href="#failed-to-read-the-metastore-db-during-table">Failed to read the metastore db during table</a></li>
+<li><a href="#failed-to-load-data-on-the-cluster">Failed to load data on the cluster</a></li>
+<li><a href="#failed-to-insert-data-on-the-cluster">Failed to insert data on the cluster</a></li>
+<li><a href="#failed-to-execute-concurrent-operations-on-table-by-multiple-workers">Failed to execute Concurrent Operations(Load,Insert,Update) on table by multiple workers</a></li>
+<li><a href="#failed-to-create-a-table-with-a-single-numeric-column">Failed to create a table with a single numeric column</a></li>
+</ul>
+<h2></h2>
 <h2>
 <a id="what-are-bad-records" class="anchor" href="#what-are-bad-records" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>What are Bad Records?</h2>
 <p>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.</p>
@@ -311,6 +378,195 @@ If wanted to control timezone of data while writing, then set cluster's time-zon
 <pre><code>cluster timezone is Asia/Shanghai
 TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
 </code></pre>
+<h2>
+<a id="getting-tablestatuslock-issues-when-loading-data" class="anchor" href="#getting-tablestatuslock-issues-when-loading-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Getting tablestatus.lock issues When loading data</h2>
+<p><strong>Symptom</strong></p>
+<pre><code>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.&lt;init&gt;(FileOutputStream.java:213)
+	at java.io.FileOutputStream.&lt;init&gt;(FileOutputStream.java:101)
+</code></pre>
+<p><strong>Possible Cause</strong>
+If you use <code>&lt;hdfs path&gt;</code> as store path when creating carbonsession, may get the errors,because the default is LOCALLOCK.</p>
+<p><strong>Procedure</strong>
+Before creating carbonsession, sets as below:</p>
+<pre><code>import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+CarbonProperties.getInstance().addProperty(CarbonCommonConstants.LOCK_TYPE, "HDFSLOCK")
+</code></pre>
+<h2>
+<a id="failed-to-load-thrift-libraries" class="anchor" href="#failed-to-load-thrift-libraries" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to load thrift libraries</h2>
+<p><strong>Symptom</strong></p>
+<p>Thrift throws following exception :</p>
+<pre><code>thrift: error while loading shared libraries:
+libthriftc.so.0: cannot open shared object file: No such file or directory
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The complete path to the directory containing the libraries is not configured correctly.</p>
+<p><strong>Procedure</strong></p>
+<p>Follow the Apache thrift docs at <a href="https://thrift.apache.org/docs/install" target=_blank rel="nofollow">https://thrift.apache.org/docs/install</a> to install thrift correctly.</p>
+<h2>
+<a id="failed-to-launch-the-spark-shell" class="anchor" href="#failed-to-launch-the-spark-shell" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to launch the Spark Shell</h2>
+<p><strong>Symptom</strong></p>
+<p>The shell prompts the following error :</p>
+<pre><code>org.apache.spark.sql.CarbonContext$$anon$$apache$spark$sql$catalyst$analysis
+$OverrideCatalog$_setter_$org$apache$spark$sql$catalyst$analysis
+$OverrideCatalog$$overrides_$e
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The Spark Version and the selected Spark Profile do not match.</p>
+<p><strong>Procedure</strong></p>
+<ol>
+<li>
+<p>Ensure your spark version and selected profile for spark are correct.</p>
+</li>
+<li>
+<p>Use the following command :</p>
+</li>
+</ol>
+<pre><code>"mvn -Pspark-2.1 -Dspark.version {yourSparkVersion} clean package"
+</code></pre>
+<p>Note :  Refrain from using "mvn clean package" without specifying the profile.</p>
+<h2>
+<a id="failed-to-execute-load-query-on-cluster" class="anchor" href="#failed-to-execute-load-query-on-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute load query on cluster</h2>
+<p><strong>Symptom</strong></p>
+<p>Load query failed with the following exception:</p>
+<pre><code>Dictionary file is locked for updation.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The carbon.properties file is not identical in all the nodes of the cluster.</p>
+<p><strong>Procedure</strong></p>
+<p>Follow the steps to ensure the carbon.properties file is consistent across all the nodes:</p>
+<ol>
+<li>
+<p>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.</p>
+</li>
+<li>
+<p>For the changes to take effect, restart the Spark cluster.</p>
+</li>
+</ol>
+<h2>
+<a id="failed-to-execute-insert-query-on-cluster" class="anchor" href="#failed-to-execute-insert-query-on-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute insert query on cluster</h2>
+<p><strong>Symptom</strong></p>
+<p>Load query failed with the following exception:</p>
+<pre><code>Dictionary file is locked for updation.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The carbon.properties file is not identical in all the nodes of the cluster.</p>
+<p><strong>Procedure</strong></p>
+<p>Follow the steps to ensure the carbon.properties file is consistent across all the nodes:</p>
+<ol>
+<li>
+<p>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.</p>
+</li>
+<li>
+<p>For the changes to take effect, restart the Spark cluster.</p>
+</li>
+</ol>
+<h2>
+<a id="failed-to-connect-to-hiveuser-with-thrift" class="anchor" href="#failed-to-connect-to-hiveuser-with-thrift" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to connect to hiveuser with thrift</h2>
+<p><strong>Symptom</strong></p>
+<p>We get the following exception :</p>
+<pre><code>Cannot connect to hiveuser.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The external process does not have permission to access.</p>
+<p><strong>Procedure</strong></p>
+<p>Ensure that the Hiveuser in mysql must allow its access to the external processes.</p>
+<h2>
+<a id="failed-to-read-the-metastore-db-during-table-creation" class="anchor" href="#failed-to-read-the-metastore-db-during-table-creation" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to read the metastore db during table creation</h2>
+<p><strong>Symptom</strong></p>
+<p>We get the following exception on trying to connect :</p>
+<pre><code>Cannot read the metastore db
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The metastore db is dysfunctional.</p>
+<p><strong>Procedure</strong></p>
+<p>Remove the metastore db from the carbon.metastore in the Spark Directory.</p>
+<h2>
+<a id="failed-to-load-data-on-the-cluster" class="anchor" href="#failed-to-load-data-on-the-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to load data on the cluster</h2>
+<p><strong>Symptom</strong></p>
+<p>Data loading fails with the following exception :</p>
+<pre><code>Data Load failure exception
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The following issue can cause the failure :</p>
+<ol>
+<li>
+<p>The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.</p>
+</li>
+<li>
+<p>Path to hdfs ddl is not configured correctly in the carbon.properties.</p>
+</li>
+</ol>
+<p><strong>Procedure</strong></p>
+<p>Follow the steps to ensure the following configuration files are consistent across all the nodes:</p>
+<ol>
+<li>
+<p>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.</p>
+<p>Note : Set the path to hdfs ddl in carbon.properties in the master node.</p>
+</li>
+<li>
+<p>For the changes to take effect, restart the Spark cluster.</p>
+</li>
+</ol>
+<h2>
+<a id="failed-to-insert-data-on-the-cluster" class="anchor" href="#failed-to-insert-data-on-the-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to insert data on the cluster</h2>
+<p><strong>Symptom</strong></p>
+<p>Insertion fails with the following exception :</p>
+<pre><code>Data Load failure exception
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>The following issue can cause the failure :</p>
+<ol>
+<li>
+<p>The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.</p>
+</li>
+<li>
+<p>Path to hdfs ddl is not configured correctly in the carbon.properties.</p>
+</li>
+</ol>
+<p><strong>Procedure</strong></p>
+<p>Follow the steps to ensure the following configuration files are consistent across all the nodes:</p>
+<ol>
+<li>
+<p>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.</p>
+<p>Note : Set the path to hdfs ddl in carbon.properties in the master node.</p>
+</li>
+<li>
+<p>For the changes to take effect, restart the Spark cluster.</p>
+</li>
+</ol>
+<h2>
+<a id="failed-to-execute-concurrent-operations-on-table-by-multiple-workers" class="anchor" href="#failed-to-execute-concurrent-operations-on-table-by-multiple-workers" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to execute Concurrent Operations on table by multiple workers</h2>
+<p><strong>Symptom</strong></p>
+<p>Execution fails with the following exception :</p>
+<pre><code>Table is locked for updation.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>Concurrency not supported.</p>
+<p><strong>Procedure</strong></p>
+<p>Worker must wait for the query execution to complete and the table to release the lock for another query execution to succeed.</p>
+<h2>
+<a id="failed-to-create-a-table-with-a-single-numeric-column" class="anchor" href="#failed-to-create-a-table-with-a-single-numeric-column" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Failed to create a table with a single numeric column</h2>
+<p><strong>Symptom</strong></p>
+<p>Execution fails with the following exception :</p>
+<pre><code>Table creation fails.
+</code></pre>
+<p><strong>Possible Cause</strong></p>
+<p>Behaviour not supported.</p>
+<p><strong>Procedure</strong></p>
+<p>A single column that can be considered as dimension is mandatory for table creation.</p>
+<script>
+// Show selected style on nav item
+$(function() { $('.b-nav__faq').addClass('selected'); });
+</script>
 </div>
 </div>
 </div>
@@ -326,4 +582,4 @@ TimeZone.setDefault(TimeZone.getTimeZone("Asia/Shanghai"))
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/file-structure-of-carbondata.html
----------------------------------------------------------------------
diff --git a/content/file-structure-of-carbondata.html b/content/file-structure-of-carbondata.html
index edebccf..b96d622 100644
--- a/content/file-structure-of-carbondata.html
+++ b/content/file-structure-of-carbondata.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -168,7 +219,7 @@
 <ul>
 <li>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.</li>
 </ul>
-<p><a href="../docs/images/carbon_data_file_structure_new.png?raw=true" target="_blank"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_file_structure_new.png?raw=true" alt="CarbonData File Structure" style="max-width:100%;"></a></p>
+<p><a href="../docs/images/carbon_data_file_structure_new.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_file_structure_new.png?raw=true" alt="CarbonData File Structure" style="max-width:100%;"></a></p>
 <ul>
 <li>File Header : It contains CarbonData file version number, list of column schema and schema updation timestamp.</li>
 <li>File Footer : it contains Number of rows, segmentinfo ,all blocklets? info and index, you can find the detail from the below diagram.</li>
@@ -176,7 +227,7 @@
 <li>Column Page Group : Data of one column and it is further divided into pages, it is guaranteed to be contiguous in file.</li>
 <li>Page : It has the data of one column and the number of row is fixed to 32000 size.</li>
 </ul>
-<p><a href="../docs/images/carbon_data_format_new.png?raw=true" target="_blank"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_format_new.png?raw=true" alt="CarbonData File Format" style="max-width:100%;"></a></p>
+<p><a href="../docs/images/carbon_data_format_new.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbon_data_format_new.png?raw=true" alt="CarbonData File Format" style="max-width:100%;"></a></p>
 <h3>
 <a id="each-page-contains-three-types-of-data" class="anchor" href="#each-page-contains-three-types-of-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Each page contains three types of data</h3>
 <ul>
@@ -184,6 +235,17 @@
 <li>Row ID Page (optional): Contains the row ID mappings used when the data page is stored as an inverted index.</li>
 <li>RLE Page (optional): Contains additional metadata used when the data page is RLE coded.</li>
 </ul>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -199,4 +261,4 @@
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>


[18/39] carbondata-site git commit: Added new page layout & updated as per new md files

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/configuration-parameters.html
----------------------------------------------------------------------
diff --git a/content/configuration-parameters.html b/content/configuration-parameters.html
index 4f4f1cd..ba73b0d 100644
--- a/content/configuration-parameters.html
+++ b/content/configuration-parameters.html
@@ -22,6 +22,7 @@
     <![endif]-->
     <script src="js/jquery.min.js"></script>
     <script src="js/bootstrap.min.js"></script>
+    <script defer src="https://use.fontawesome.com/releases/v5.0.8/js/all.js"></script>
 
 
 </head>
@@ -67,7 +68,7 @@
                                    target="_blank">Release Archive</a></li>
                         </ul>
                     </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li><a href="documentation.html" class="active">Documentation</a></li>
                     <li class="dropdown">
                         <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
                            aria-expanded="false">Community <span class="caret"></span></a>
@@ -152,7 +153,57 @@
 <section><!-- Dashboard nav -->
     <div class="container-fluid q">
         <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
+            <div class="verticalnavbar">
+                <nav class="b-sticky-nav">
+                    <div class="nav-scroller">
+                        <div class="nav__inner">
+                            <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
+                            <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
+                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
+                                <a class="nav__item nav__sub__item" href="./ddl-of-carbondata.html">DDL</a>
+                                <a class="nav__item nav__sub__item" href="./dml-of-carbondata.html">DML</a>
+                                <a class="nav__item nav__sub__item" href="./streaming-guide.html">Streaming</a>
+                                <a class="nav__item nav__sub__item" href="./configuration-parameters.html">Configuration</a>
+                                <a class="nav__item nav__sub__item" href="./datamap-developer-guide.html">Datamaps</a>
+                                <a class="nav__item nav__sub__item" href="./supported-data-types-in-carbondata.html">Data Types</a>
+                            </div>
+
+                            <div class="nav__item nav__item__with__subs">
+                                <a class="b-nav__datamap nav__item nav__sub__anchor" href="./datamap-management.html">DataMaps</a>
+                                <a class="nav__item nav__sub__item" href="./bloomfilter-datamap-guide.html">Bloom Filter</a>
+                                <a class="nav__item nav__sub__item" href="./lucene-datamap-guide.html">Lucene</a>
+                                <a class="nav__item nav__sub__item" href="./preaggregate-datamap-guide.html">Pre-Aggregate</a>
+                                <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
+                            </div>
+
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
+                            <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
+                            <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
+                            <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
+                            <a class="b-nav__security nav__item" href="./security.html">Security</a>
+                            <a class="b-nav__release nav__item" href="./release-guide.html">Release Guide</a>
+                        </div>
+                    </div>
+                    <div class="navindicator">
+                        <div class="b-nav__intro navindicator__item"></div>
+                        <div class="b-nav__quickstart navindicator__item"></div>
+                        <div class="b-nav__uses navindicator__item"></div>
+                        <div class="b-nav__docs navindicator__item"></div>
+                        <div class="b-nav__datamap navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
+                        <div class="b-nav__api navindicator__item"></div>
+                        <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__faq navindicator__item"></div>
+                        <div class="b-nav__contri navindicator__item"></div>
+                        <div class="b-nav__security navindicator__item"></div>
+                    </div>
+                </nav>
+            </div>
+            <div class="mdcontent">
                 <section>
                     <div style="padding:10px 15px;">
                         <div id="viewpage" name="viewpage">
@@ -161,18 +212,18 @@
                                     <div>
 <h1>
 <a id="configuring-carbondata" class="anchor" href="#configuring-carbondata" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configuring CarbonData</h1>
-<p>This tutorial guides you through the advanced configurations of CarbonData :</p>
+<p>This guide explains the configurations that can be used to tune CarbonData to achieve better performance.Some of the properties can be set dynamically and are explained in the section Dynamic Configuration In CarbonData Using SET-RESET.Most of the properties that control the internal settings have reasonable default values.They are listed along with the properties along with explanation.</p>
 <ul>
 <li><a href="#system-configuration">System Configuration</a></li>
-<li><a href="#performance-configuration">Performance Configuration</a></li>
-<li><a href="#miscellaneous-configuration">Miscellaneous Configuration</a></li>
-<li><a href="#spark-configuration">Spark Configuration</a></li>
+<li><a href="#data-loading-configuration">Data Loading Configuration</a></li>
+<li><a href="#compaction-configuration">Compaction Configuration</a></li>
+<li><a href="#query-configuration">Query Configuration</a></li>
+<li><a href="#data-mutation-configuration">Data Mutation Configuration</a></li>
 <li><a href="#dynamic-configuration-in-carbondata-using-set-reset">Dynamic Configuration In CarbonData Using SET-RESET</a></li>
 </ul>
 <h2>
 <a id="system-configuration" class="anchor" href="#system-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>System Configuration</h2>
 <p>This section provides the details of all the configurations required for the CarbonData System.</p>
-<p><b></b></p><p align="center">System Configuration in carbon.properties</p>
 <table>
 <thead>
 <tr>
@@ -184,411 +235,303 @@
 <tbody>
 <tr>
 <td>carbon.storelocation</td>
-<td></td>
-<td>Location where CarbonData will create the store, and write the data in its own format. If not specified then it takes spark.sql.warehouse.dir path. NOTE: Store location should be in HDFS.</td>
+<td>spark.sql.warehouse.dir property value</td>
+<td>Location where CarbonData will create the store, and write the data in its custom format. If not specified,the path defaults to spark.sql.warehouse.dir property. NOTE: Store location should be in HDFS.</td>
 </tr>
 <tr>
 <td>carbon.ddl.base.hdfs.url</td>
-<td></td>
-<td>This property is used to configure the HDFS relative path, the path configured in carbon.ddl.base.hdfs.url will be appended to the HDFS path configured in fs.defaultFS. If this path is configured, then user need not pass the complete path while dataload. For example: If absolute path of the csv file is hdfs://10.18.101.155:54310/data/cnbc/2016/xyz.csv, the path "hdfs://10.18.101.155:54310" will come from property fs.defaultFS and user can configure the /data/cnbc/ as carbon.ddl.base.hdfs.url. Now while dataload user can specify the csv path as /2016/xyz.csv.</td>
+<td>(none)</td>
+<td>To simplify and shorten the path to be specified in DDL/DML commands, this property is supported.This property is used to configure the HDFS relative path, the path configured in carbon.ddl.base.hdfs.url will be appended to the HDFS path configured in fs.defaultFS of core-site.xml. If this path is configured, then user need not pass the complete path while dataload. For example: If absolute path of the csv file is hdfs://10.18.101.155:54310/data/cnbc/2016/xyz.csv, the path "hdfs://10.18.101.155:54310" will come from property fs.defaultFS and user can configure the /data/cnbc/ as carbon.ddl.base.hdfs.url. Now while dataload user can specify the csv path as /2016/xyz.csv.</td>
 </tr>
 <tr>
 <td>carbon.badRecords.location</td>
-<td></td>
-<td>Path where the bad records are stored.</td>
-</tr>
-<tr>
-<td>carbon.data.file.version</td>
-<td>V3</td>
-<td>If this parameter value is set to 1, then CarbonData will support the data load which is in old format(0.x version). If the value is set to 2(1.x onwards version), then CarbonData will support the data load of new format only.</td>
+<td>(none)</td>
+<td>CarbonData can detect the records not conforming to defined table schema and isolate them as bad records.This property is used to specify where to store such bad records.</td>
 </tr>
 <tr>
 <td>carbon.streaming.auto.handoff.enabled</td>
 <td>true</td>
-<td>If this parameter value is set to true, auto trigger handoff function will be enabled.</td>
+<td>CarbonData supports storing of streaming data.To have high throughput for streaming, the data is written in Row format which is highly optimized for write, but performs poorly for query.When this property is true and when the streaming data size reaches <em><strong>carbon.streaming.segment.max.size</strong></em>, CabonData will automatically convert the data to columnar format and optimize it for faster querying.<strong>NOTE:</strong> It is not recommended to keep the default value which is true.</td>
 </tr>
 <tr>
 <td>carbon.streaming.segment.max.size</td>
 <td>1024000000</td>
-<td>This parameter defines the maximum size of the streaming segment. Setting this parameter to appropriate value will avoid impacting the streaming ingestion. The value is in bytes.</td>
+<td>CarbonData writes streaming data in row format which is optimized for high write throughput.This property defines the maximum size of data to be held is row format, beyond which it will be converted to columnar format in order to support high performane query, provided <em><strong>carbon.streaming.auto.handoff.enabled</strong></em> is true. <strong>NOTE:</strong> Setting higher value will impact the streaming ingestion. The value has to be configured in bytes.</td>
 </tr>
 <tr>
 <td>carbon.query.show.datamaps</td>
 <td>true</td>
-<td>If this parameter value is set to true, show tables command will list all the tables including datatmaps(eg: Preaggregate table), else datamaps will be excluded from the table list.</td>
+<td>CarbonData stores datamaps as independent tables so as to allow independent maintenance to some extent.When this property is true,which is by default, show tables command will list all the tables including datatmaps(eg: Preaggregate table), else datamaps will be excluded from the table list.<strong>NOTE:</strong>  It is generally not required for the user to do any maintenance operations on these tables and hence not required to be seen.But it is shown by default so that user or admin can get clear understanding of the system for capacity planning.</td>
 </tr>
 <tr>
 <td>carbon.segment.lock.files.preserve.hours</td>
 <td>48</td>
-<td>This property value indicates the number of hours the segment lock files will be preserved after dataload. These lock files will be deleted with the clean command after the configured number of hours.</td>
+<td>In order to support parallel data loading onto the same table, CarbonData sequences(locks) at the granularity of segments.Operations affecting the segment(like IUD, alter) are blocked from parallel operations.This property value indicates the number of hours the segment lock files will be preserved after dataload. These lock files will be deleted with the clean command after the configured number of hours.</td>
+</tr>
+<tr>
+<td>carbon.timestamp.format</td>
+<td>yyyy-MM-dd HH:mm:ss</td>
+<td>CarbonData can understand data of timestamp type and process it in special manner.It can be so that the format of Timestamp data is different from that understood by CarbonData by default.This configuration allows users to specify the format of Timestamp in their data.</td>
+</tr>
+<tr>
+<td>carbon.lock.type</td>
+<td>LOCALLOCK</td>
+<td>This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: 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: 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 HDFS supports file based locking.</td>
+</tr>
+<tr>
+<td>carbon.lock.path</td>
+<td>TABLEPATH</td>
+<td>This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3.</td>
 </tr>
 <tr>
 <td>carbon.unsafe.working.memory.in.mb</td>
 <td>512</td>
-<td>Specifies the size of executor unsafe working memory. Used for sorting data, storing column pages,etc. This value is expressed in MB.</td>
+<td>CarbonData supports storing data in off-heap memory for certain operations during data loading and query.This helps to avoid the Java GC and thereby improve the overall performance.The Minimum value recommeded is 512MB.Any value below this is reset to default value of 512MB.<strong>NOTE:</strong> The below formulas explain how to arrive at the off-heap size required.Memory Required For Data Loading:(<em>carbon.number.of.cores.while.loading</em>) * (Number of tables to load in parallel) * (<em>offheap.sort.chunk.size.inmb</em> + <em>carbon.blockletgroup.size.in.mb</em> + <em>carbon.blockletgroup.size.in.mb</em>/3.5 ). Memory required for Query:SPARK_EXECUTOR_INSTANCES * (<em>carbon.blockletgroup.size.in.mb</em> + <em>carbon.blockletgroup.size.in.mb</em> * 3.5) * spark.executor.cores</td>
 </tr>
 <tr>
-<td>carbon.unsafe.driver.working.memory.in.mb</td>
-<td>512</td>
-<td>Specifies the size of driver unsafe working memory. Used for storing block or blocklet datamap cache. If not configured then carbon.unsafe.working.memory.in.mb value is considered. This value is expressed in MB.</td>
+<td>carbon.update.sync.folder</td>
+<td>/tmp/carbondata</td>
+<td>CarbonData maintains last modification time entries in modifiedTime.htmlt to determine the schema changes and reload only when necessary.This configuration specifies the path where the file needs to be written.</td>
+</tr>
+<tr>
+<td>carbon.invisible.segments.preserve.count</td>
+<td>200</td>
+<td>CarbonData maintains each data load entry in tablestatus file. The entries from this file are not deleted for those segments that are compacted or dropped, but are made invisible.If the number of data loads are very high, the size and number of entries in tablestatus file can become too many causing unnecessary reading of all data.This configuration specifies the number of segment entries to be maintained afte they are compacted or dropped.Beyond this, the entries are moved to a separate history tablestatus file.<strong>NOTE:</strong> The entries in tablestatus file help to identify the operations performed on CarbonData table and is also used for checkpointing during various data manupulation operations.This is similar to AUDIT file maintaining all the operations and its status.Hence the entries are never deleted but moved to a separate history file.</td>
+</tr>
+<tr>
+<td>carbon.lock.retries</td>
+<td>3</td>
+<td>CarbonData ensures consistency of operations by blocking certain operations from running in parallel.In order to block the operations from running in parallel, lock is obtained on the table.This configuration specifies the maximum number of retries to obtain the lock for any operations other than load.<strong>NOTE:</strong> Data manupulation operations like Compaction,UPDATE,DELETE  or LOADING,UPDATE,DELETE are not allowed to run in parallel.How ever data loading can happen in parallel to compaction.</td>
+</tr>
+<tr>
+<td>carbon.lock.retry.timeout.sec</td>
+<td>5</td>
+<td>Specifies the interval between the retries to obtain the lock for any operation other than load.<strong>NOTE:</strong> Refer to <em><strong>carbon.lock.retries</strong></em> for understanding why CarbonData uses locks for operations.</td>
 </tr>
 </tbody>
 </table>
 <h2>
-<a id="performance-configuration" class="anchor" href="#performance-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Performance Configuration</h2>
-<p>This section provides the details of all the configurations required for CarbonData Performance Optimization.</p>
-<p><b></b></p><p align="center">Performance Configuration in carbon.properties</p>
-<ul>
-<li><strong>Data Loading Configuration</strong></li>
-</ul>
+<a id="data-loading-configuration" class="anchor" href="#data-loading-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Loading Configuration</h2>
 <table>
 <thead>
 <tr>
 <th>Parameter</th>
 <th>Default Value</th>
 <th>Description</th>
-<th>Range</th>
 </tr>
 </thead>
 <tbody>
 <tr>
 <td>carbon.number.of.cores.while.loading</td>
 <td>2</td>
-<td>Number of cores to be used while loading data.</td>
-<td></td>
+<td>Number of cores to be used while loading data.This also determines the number of threads to be used to read the input files (csv) in parallel.<strong>NOTE:</strong> This configured value is used in every data loading step to parallelize the operations. Configuring a higher value can lead to increased early thread pre-emption by OS and there by reduce the overall performance.</td>
 </tr>
 <tr>
 <td>carbon.sort.size</td>
 <td>100000</td>
-<td>Record count to sort and write intermediate files to temp.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.max.driver.lru.cache.size</td>
-<td>-1</td>
-<td>Max LRU cache size upto which data will be loaded at the driver side. This value is expressed in MB. Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.max.executor.lru.cache.size</td>
-<td>-1</td>
-<td>Max LRU cache size upto which data will be loaded at the executor side. This value is expressed in MB. Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted. If this parameter is not configured, then the carbon.max.driver.lru.cache.size value will be considered.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.merge.sort.prefetch</td>
-<td>true</td>
-<td>Enable prefetch of data during merge sort while reading data from sort temp files in data loading.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.insert.persist.enable</td>
-<td>false</td>
-<td>Enabling this parameter considers persistent data. If we are executing insert into query from source table using select statement &amp; loading the same source table concurrently, when select happens on source table during the data load, it gets new record for which dictionary is not generated, so there will be inconsistency. To avoid this condition we can persist the dataframe into MEMORY_AND_DISK(default value) and perform insert into operation. By default this value will be false because no need to persist the dataframe in all cases. If user wants to run load and insert queries on source table concurrently then user can enable this parameter.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.insert.storage.level</td>
-<td>MEMORY_AND_DISK</td>
-<td>Which storage level to persist dataframe when 'carbon.insert.persist.enable'=true, if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. <a href="http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence" rel="nofollow">See detail</a>.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.update.persist.enable</td>
-<td>true</td>
-<td>Enabling this parameter considers persistent data. Enabling this will reduce the execution time of UPDATE operation.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.update.storage.level</td>
-<td>MEMORY_AND_DISK</td>
-<td>Which storage level to persist dataframe when 'carbon.update.persist.enable'=true, if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. <a href="http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence" rel="nofollow">See detail</a>.</td>
-<td></td>
+<td>Number of records to hold in memory to sort and write intermediate temp files.<strong>NOTE:</strong> Memory required for data loading increases with increase in configured value as each thread would cache configured number of records.</td>
 </tr>
 <tr>
 <td>carbon.global.sort.rdd.storage.level</td>
 <td>MEMORY_ONLY</td>
-<td>Which storage level to persist rdd when loading data with 'sort_scope'='global_sort', if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. <a href="http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence" rel="nofollow">See detail</a>.</td>
-<td></td>
+<td>Storage level to persist dataset of RDD/dataframe when loading data with 'sort_scope'='global_sort', if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. <a href="http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence" rel="nofollow">See detail</a>.</td>
 </tr>
 <tr>
 <td>carbon.load.global.sort.partitions</td>
 <td>0</td>
-<td>The Number of partitions to use when shuffling data for sort. If user don't configurate or configurate it less than 1, it uses the number of map tasks as reduce tasks. In general, we recommend 2-3 tasks per CPU core in your cluster.</td>
-<td></td>
+<td>The Number of partitions to use when shuffling data for sort. Default value 0 means to use same number of map tasks as reduce tasks.<strong>NOTE:</strong> In general, it is recommended to have 2-3 tasks per CPU core in your cluster.</td>
 </tr>
 <tr>
 <td>carbon.options.bad.records.logger.enable</td>
 <td>false</td>
-<td>Whether to create logs with details about bad records.</td>
-<td></td>
+<td>CarbonData can identify the records that are not conformant to schema and isolate them as bad records.Enabling this configuration will make CarbonData to log such bad records.<strong>NOTE:</strong> If the input data contains many bad records, logging them will slow down the over all data loading throughput.The data load operation status would depend on the configuration in <em><strong>carbon.bad.records.action</strong></em>.</td>
 </tr>
 <tr>
 <td>carbon.bad.records.action</td>
-<td>FORCE</td>
-<td>This property can have four types of actions for bad records FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found.</td>
-<td></td>
+<td>FAIL</td>
+<td>CarbonData in addition to identifying the bad records, can take certain actions on such data.This configuration can have four types of actions for bad records namely FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found.</td>
 </tr>
 <tr>
 <td>carbon.options.is.empty.data.bad.record</td>
 <td>false</td>
-<td>If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa.</td>
-<td></td>
+<td>Based on the business scenarios, empty("" or '' or ,,) data can be valid or invalid. This configuration controls how empty data should be treated by CarbonData. If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa.</td>
 </tr>
 <tr>
 <td>carbon.options.bad.record.path</td>
-<td></td>
-<td>Specifies the HDFS path where bad records are stored. By default the value is Null. This path must to be configured by the user if bad record logger is enabled or bad record action redirect.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.enable.vector.reader</td>
-<td>true</td>
-<td>This parameter increases the performance of select queries as it fetch columnar batch of size 4*1024 rows instead of fetching data row by row.</td>
-<td></td>
+<td>(none)</td>
+<td>Specifies the HDFS path where bad records are to be stored. By default the value is Null. This path must to be configured by the user if <em><strong>carbon.options.bad.records.logger.enable</strong></em> is <strong>true</strong> or <em><strong>carbon.bad.records.action</strong></em> is <strong>REDIRECT</strong>.</td>
 </tr>
 <tr>
 <td>carbon.blockletgroup.size.in.mb</td>
-<td>64 MB</td>
-<td>The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of the blocklet group. Higher value results in better sequential IO access.The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB).</td>
-<td></td>
+<td>64</td>
+<td>Please refer to <a href="./file-structure-of-carbondata.html">file-structure-of-carbondata</a> to understand the storage format of CarbonData.The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of each blocklet group. Higher value results in better sequential IO access.The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB).<strong>NOTE:</strong> Configuring a higher value might lead to poor performance as an entire blocklet group will have to read into memory before processing.For filter queries with limit, it is <strong>not advisable</strong> to have a bigger blocklet size.For Aggregation queries which need to return more number of rows,bigger blocklet size is advisable.</td>
 </tr>
 <tr>
-<td>carbon.task.distribution</td>
-<td>block</td>
-<td>
-<strong>block</strong>: Setting this value will launch one task per block. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. <strong>custom</strong>: Setting this value will group the blocks and distribute it uniformly to the available resources in the cluster. This enhances the query performance but not suggested in case of concurrent queries and queries having big shuffling scenarios. <strong>blocklet</strong>: Setting this value will launch one task per blocklet. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. <strong>merge_small_files</strong>: Setting this value will merge all the small partitions to a size of (128 MB is the default value of "spark.sql.files.maxPartitionBytes",it is configurable) during querying. The small partitions are combined to a map task to reduce the number of read task. This enhances the performance.</td>
-<td></td>
-</tr>
-<tr>
-<td>carbon.load.sortmemory.spill.percentage</td>
-<td>0</td>
-<td>If we use unsafe memory during data loading, this configuration will be used to control the behavior of spilling inmemory pages to disk. Internally in Carbondata, during sorting carbondata will sort data in pages and add them in unsafe memory. If the memory is insufficient, carbondata will spill the pages to disk and generate sort temp file. This configuration controls how many pages in memory will be spilled to disk based size. The size can be calculated by multiplying this configuration value with 'carbon.sort.storage.inmemory.size.inmb'. For example, default value 0 means that no pages in unsafe memory will be spilled and all the newly sorted data will be spilled to disk; Value 50 means that if the unsafe memory is insufficient, about half of pages in the unsafe memory will be spilled to disk while value 100 means that almost all pages in unsafe memory will be spilled. <strong>Note</strong>: This configuration only works for 'LOCAL_SORT' and 'BATCH_SORT' and the actual spilli
 ng behavior may slightly be different in each data loading.</td>
-<td>Integer values between 0 and 100</td>
-</tr>
-</tbody>
-</table>
-<ul>
-<li><strong>Compaction Configuration</strong></li>
-</ul>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Default Value</th>
-<th>Description</th>
-<th>Range</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>carbon.number.of.cores.while.compacting</td>
-<td>2</td>
-<td>Number of cores which are used to write data during compaction.</td>
-<td></td>
+<td>carbon.sort.file.write.buffer.size</td>
+<td>16384</td>
+<td>CarbonData sorts and writes data to intermediate files to limit the memory usage.This configuration determines the buffer size to be used for reading and writing such files. <strong>NOTE:</strong> This configuration is useful to tune IO and derive optimal performance.Based on the OS and underlying harddisk type, these values can significantly affect the overall performance.It is ideal to tune the buffersize equivalent to the IO buffer size of the OS.Recommended range is between 10240 to 10485760 bytes.</td>
 </tr>
 <tr>
-<td>carbon.compaction.level.threshold</td>
-<td>4, 3</td>
-<td>This property is for minor compaction which decides how many segments to be merged. Example: If it is set as 2, 3 then minor compaction will be triggered for every 2 segments. 3 is the number of level 1 compacted segment which is further compacted to new segment.</td>
-<td>Valid values are from 0-100.</td>
+<td>carbon.sort.intermediate.files.limit</td>
+<td>20</td>
+<td>CarbonData sorts and writes data to intermediate files to limit the memory usage.Before writing the target carbondat file, the data in these intermediate files needs to be sorted again so as to ensure the entire data in the data load is sorted.This configuration determines the minimum number of intermediate files after which merged sort is applied on them sort the data.<strong>NOTE:</strong> Intermediate merging happens on a separate thread in the background.Number of threads used is determined by <em><strong>carbon.merge.sort.reader.thread</strong></em>.Configuring a low value will cause more time to be spent in merging these intermediate merged files which can cause more IO.Configuring a high value would cause not to use the idle threads to do intermediate sort merges.Range of recommended values are between 2 and 50</td>
 </tr>
 <tr>
-<td>carbon.major.compaction.size</td>
-<td>1024</td>
-<td>Major compaction size can be configured using this parameter. Sum of the segments which is below this threshold will be merged. This value is expressed in MB.</td>
-<td></td>
+<td>carbon.csv.read.buffersize.byte</td>
+<td>1048576</td>
+<td>CarbonData uses Hadoop InputFormat to read the csv files.This configuration value is used to pass buffer size as input for the Hadoop MR job when reading the csv files.This value is configured in bytes.<strong>NOTE:</strong> Refer to <em><strong>org.apache.hadoop.mapreduce.InputFormat</strong></em> documentation for additional information.</td>
 </tr>
 <tr>
-<td>carbon.horizontal.compaction.enable</td>
-<td>true</td>
-<td>This property is used to turn ON/OFF horizontal compaction. After every DELETE and UPDATE statement, horizontal compaction may occur in case the delta (DELETE/ UPDATE) files becomes more than specified threshold.</td>
-<td></td>
+<td>carbon.merge.sort.reader.thread</td>
+<td>3</td>
+<td>CarbonData sorts and writes data to intermediate files to limit the memory usage.When the intermediate files reaches <em><strong>carbon.sort.intermediate.files.limit</strong></em> the files will be merged,the number of threads specified in this configuration will be used to read the intermediate files for performing merge sort.<strong>NOTE:</strong> Refer to <em><strong>carbon.sort.intermediate.files.limit</strong></em> for operation description.Configuring less  number of threads can cause merging to slow down over loading process where as configuring more number of threads can cause thread contention with threads in other data loading steps.Hence configure a fraction of <em><strong>carbon.number.of.cores.while.loading</strong></em>.</td>
 </tr>
 <tr>
-<td>carbon.horizontal.UPDATE.compaction.threshold</td>
-<td>1</td>
-<td>This property specifies the threshold limit on number of UPDATE delta files within a segment. In case the number of delta files goes beyond the threshold, the UPDATE delta files within the segment becomes eligible for horizontal compaction and compacted into single UPDATE delta file.</td>
-<td>Values between 1 to 10000.</td>
+<td>carbon.concurrent.lock.retries</td>
+<td>100</td>
+<td>CarbonData supports concurrent data loading onto same table.To ensure the loading status is correctly updated into the system,locks are used to sequence the status updation step.This configuration specifies the maximum number of retries to obtain the lock for updating the load status.<strong>NOTE:</strong> This value is high as more number of concurrent loading happens,more the chances of not able to obtain the lock when tried.Adjust this value according to the number of concurrent loading to be supported by the system.</td>
 </tr>
 <tr>
-<td>carbon.horizontal.DELETE.compaction.threshold</td>
+<td>carbon.concurrent.lock.retry.timeout.sec</td>
 <td>1</td>
-<td>This property specifies the threshold limit on number of DELETE delta files within a block of a segment. In case the number of delta files goes beyond the threshold, the DELETE delta files for the particular block of the segment becomes eligible for horizontal compaction and compacted into single DELETE delta file.</td>
-<td>Values between 1 to 10000.</td>
+<td>Specifies the interval between the retries to obtain the lock for concurrent operations.<strong>NOTE:</strong> Refer to <em><strong>carbon.concurrent.lock.retries</strong></em> for understanding why CarbonData uses locks during data loading operations.</td>
 </tr>
 <tr>
-<td>carbon.update.segment.parallelism</td>
-<td>1</td>
-<td>This property specifies the parallelism for each segment during update. If there are segments that contain too many records to update and the spark job encounter data-spill related errors, it is better to increase this property value. It is recommended to set this value to a multiple of the number of executors for balance.</td>
-<td>Values between 1 to 1000.</td>
+<td>carbon.skip.empty.line</td>
+<td>false</td>
+<td>The csv files givent to CarbonData for loading can contain empty lines.Based on the business scenario, this empty line might have to be ignored or needs to be treated as NULL value for all columns.In order to define this business behavior, this configuration is provided.<strong>NOTE:</strong> In order to consider NULL values for non string columns and continue with data load, <em><strong>carbon.bad.records.action</strong></em> need to be set to <strong>FORCE</strong>;else data load will be failed as bad records encountered.</td>
 </tr>
 <tr>
-<td>carbon.merge.index.in.segment</td>
+<td>carbon.enable.calculate.size</td>
 <td>true</td>
-<td>This property is used to merge all carbon index files (.carbonindex) inside a segment to a single carbon index merge file (.carbonindexmerge).</td>
-<td>Values true or false</td>
+<td>
+<strong>For Load Operation</strong>: Setting this property calculates the size of the carbon data file (.carbondata) and carbon index file (.carbonindex) for every load and updates the table status file. <strong>For Describe Formatted</strong>: Setting this property calculates the total size of the carbon data files and carbon index files for the respective table and displays in describe formatted command.<strong>NOTE:</strong> This is useful to determine the overall size of the carbondata table and also get an idea of how the table is growing in order to take up other backup strategy decisions.</td>
 </tr>
-</tbody>
-</table>
-<ul>
-<li><strong>Query Configuration</strong></li>
-</ul>
-<table>
-<thead>
 <tr>
-<th>Parameter</th>
-<th>Default Value</th>
-<th>Description</th>
-<th>Range</th>
+<td>carbon.cutOffTimestamp</td>
+<td>(none)</td>
+<td>CarbonData has capability to generate the Dictionary values for the timestamp columns from the data itself without the need to store the computed dictionary values. This configuration sets the start date for calculating the timestamp. Java counts the number of milliseconds from start of "1970-01-01 00:00:00". This property is used to customize the start of position. For example "2000-01-01 00:00:00". <strong>NOTE:</strong> The date must be in the form <em><strong>carbon.timestamp.format</strong></em>. CarbonData supports storing data for upto 68 years.For example, if the cut-off time is 1970-01-01 05:30:00, then data upto 2038-01-01 05:30:00 will be supported by CarbonData.</td>
 </tr>
-</thead>
-<tbody>
 <tr>
-<td>carbon.number.of.cores</td>
-<td>4</td>
-<td>Number of cores to be used while querying.</td>
-<td></td>
+<td>carbon.timegranularity</td>
+<td>SECOND</td>
+<td>The configuration is used to specify the data granularity level such as DAY, HOUR, MINUTE, or SECOND.This helps to store more than 68 years of data into CarbonData.</td>
 </tr>
 <tr>
-<td>carbon.enable.quick.filter</td>
+<td>carbon.use.local.dir</td>
 <td>false</td>
-<td>Improves the performance of filter query.</td>
-<td></td>
+<td>CarbonData during data loading, writes files to local temp directories before copying the files to HDFS.This configuration is used to specify whether CarbonData can write locally to tmp directory of the container or to the YARN application directory.</td>
 </tr>
-</tbody>
-</table>
-<h2>
-<a id="miscellaneous-configuration" class="anchor" href="#miscellaneous-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Miscellaneous Configuration</h2>
-<p><b></b></p><p align="center">Extra Configuration in carbon.properties</p>
-<ul>
-<li><strong>Time format for CarbonData</strong></li>
-</ul>
-<table>
-<thead>
 <tr>
-<th>Parameter</th>
-<th>Default Format</th>
-<th>Description</th>
+<td>carbon.use.multiple.temp.dir</td>
+<td>false</td>
+<td>When multiple disks are present in the system, YARN is generally configured with multiple disks to be used as temp directories for managing the containers.This configuration specifies whether to use multiple YARN local directories during data loading for disk IO load balancing.Enable <em><strong>carbon.use.local.dir</strong></em> for this configuration to take effect.<strong>NOTE:</strong> Data Loading is an IO intensive operation whose performance can be limited by the disk IO threshold, particularly during multi table concurrent data load.Configuring this parameter, balances the disk IO across multiple disks there by improving the over all load performance.</td>
 </tr>
-</thead>
-<tbody>
 <tr>
-<td>carbon.timestamp.format</td>
-<td>yyyy-MM-dd HH:mm:ss</td>
-<td>Timestamp format of input data used for timestamp data type.</td>
+<td>carbon.sort.temp.compressor</td>
+<td>(none)</td>
+<td>CarbonData writes every <em><strong>carbon.sort.size</strong></em> number of records to intermediate temp files during data loading to ensure memory footprint is within limits.These temporary files cab be compressed and written in order to save the storage space.This configuration specifies the name of compressor to be used to compress the intermediate sort temp files during sort procedure in data loading.The valid values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files.<strong>NOTE:</strong> Compressor will be useful if you encounter disk bottleneck.Since the data needs to be compressed and decompressed,it involves additional CPU cycles,but is compensated by the high IO throughput due to less data to be written or read from the disks.</td>
 </tr>
-</tbody>
-</table>
-<ul>
-<li><strong>Dataload Configuration</strong></li>
-</ul>
-<table>
-<thead>
 <tr>
-<th>Parameter</th>
-<th>Default Value</th>
-<th>Description</th>
+<td>carbon.load.skewedDataOptimization.enabled</td>
+<td>false</td>
+<td>During data loading,CarbonData would divide the number of blocks equally so as to ensure all executors process same number of blocks.This mechanism satisfies most of the scenarios and ensures maximum parallel processing for optimal data loading performance.In some business scenarios, there might be scenarios where the size of blocks vary significantly and hence some executors would have to do more work if they get blocks containing more data. This configuration enables size based block allocation strategy for data loading.When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data.<strong>NOTE:</strong> This configuration is useful if the size of your input data files varies widely, say 1MB~1GB.For this configuration to work effectively,knowing the data pattern and size is important and necessary.</td>
 </tr>
-</thead>
-<tbody>
 <tr>
-<td>carbon.sort.file.write.buffer.size</td>
-<td>16384</td>
-<td>File write buffer size used during sorting. Minimum allowed buffer size is 10240 byte and Maximum allowed buffer size is 10485760 byte.</td>
+<td>carbon.load.min.size.enabled</td>
+<td>false</td>
+<td>During Data Loading, CarbonData would divide the number of files among the available executors to parallelize the loading operation.When the input data files are very small, this action causes to generate many small carbondata files.This configuration determines whether to enable node minumun input data size allocation strategy for data loading.It will make sure that the node load the minimum amount of data there by reducing number of carbondata files.<strong>NOTE:</strong> This configuration is useful if the size of the input data files are very small, like 1MB~256MB.Refer to <em><strong>load_min_size_inmb</strong></em> to configure the minimum size to be considered for splitting files among executors.</td>
 </tr>
 <tr>
-<td>carbon.lock.type</td>
-<td>LOCALLOCK</td>
-<td>This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: 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: 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 HDFS supports file based locking.</td>
+<td>enable.data.loading.statistics</td>
+<td>false</td>
+<td>CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues.This configuration when made <em><strong>true</strong></em> would log additional data loading statistics information to more accurately locate the issues being debugged.<strong>NOTE:</strong> Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time.It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately.Also extensive logging is an increased IO operation and hence over all data loading performance might get reduced.Therefore it is recommened to enable this configuration only for the duration of debugging.</td>
 </tr>
 <tr>
-<td>carbon.lock.path</td>
-<td>TABLEPATH</td>
-<td>Locks on the files are used to prevent concurrent operation from modifying the same files. This</td>
+<td>carbon.dictionary.chunk.size</td>
+<td>10000</td>
+<td>CarbonData generates dictionary keys and writes them to separate dictionary file during data loading.To optimize the IO, this configuration determines the number of dictionary keys to be persisted to dictionary file at a time.<strong>NOTE:</strong> Writing to file also serves as a commit point to the dictionary generated.Increasing more values in memory causes more data loss during system or application failure.It is advised to alter this configuration judiciously.</td>
 </tr>
 <tr>
-<td>configuration specifies the path where lock files have to be created. Recommended to configure</td>
-<td></td>
-<td></td>
+<td>dictionary.worker.threads</td>
+<td>1</td>
+<td>CarbonData supports Optimized data loading by relying on a dictionary server.Dictionary server helps  to maintain dictionary values independent of the data loading and there by avoids reading the same input data multiples times.This configuration determines the number of concurrent dictionary generation or request that needs to be served by the dictionary server.<strong>NOTE:</strong> This configuration takes effect when <em><strong>carbon.options.single.pass</strong></em> is configured as true.Please refer to <em>carbon.options.single.pass</em>to understand how dictionary server optimizes data loading.</td>
 </tr>
 <tr>
-<td>HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3.</td>
-<td></td>
-<td></td>
+<td>enable.unsafe.sort</td>
+<td>true</td>
+<td>CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations.This configuration enables to use unsafe functions in CarbonData.<strong>NOTE:</strong> For operations like data loading, which generates more short lived Java objects, Java GC can be a bottle neck.Using unsafe can overcome the GC overhead and improve the overall performance.</td>
 </tr>
 <tr>
-<td>
-<strong>Note:</strong> If this property is not set to HDFS location for S3 store, then there is a possibility</td>
-<td></td>
-<td></td>
+<td>enable.offheap.sort</td>
+<td>true</td>
+<td>CarbonData supports storing data in off-heap memory for certain operations during data loading and query.This helps to avoid the Java GC and thereby improve the overall performance.This configuration enables using off-heap memory for sorting of data during data loading.<strong>NOTE:</strong>  <em><strong>enable.unsafe.sort</strong></em> configuration needs to be configured to true for using off-heap</td>
 </tr>
 <tr>
-<td>of data corruption because multiple data manipulation calls might try to update the status file</td>
-<td></td>
-<td></td>
+<td>enable.inmemory.merge.sort</td>
+<td>false</td>
+<td>CarbonData sorts and writes data to intermediate files to limit the memory usage.These intermediate files needs to be sorted again using merge sort before writing to the final carbondata file.Performing merge sort in memory would increase the sorting performance at the cost of increased memory footprint. This Configuration specifies to do in-memory merge sort or to do file based merge sort.</td>
 </tr>
 <tr>
-<td>and as lock is not acquired before updation data might get overwritten.</td>
-<td></td>
-<td></td>
+<td>carbon.load.sort.scope</td>
+<td>LOCAL_SORT</td>
+<td>CarbonData can support various sorting options to match the balance between load and query performance.LOCAL_SORT:All the data given to an executor in the single load is fully sorted and written to carondata files.Data loading performance is reduced a little as the entire data needs to be sorted in the executor.BATCH_SORT:Sorts the data in batches of configured size and writes to carbondata files.Data loading performance increases as the entire data need not be sorted.But query performance will get reduced due to false positives in block pruning and also due to more number of carbondata files written.Due to more number of carbondata files, if identified blocks &gt; cluster parallelism, query performance and concurrency will get reduced.GLOBAL SORT:Entire data in the data load is fully sorted and written to carbondata files.Data loading perfromance would get reduced as the entire data needs to be sorted.But the query performance increases significantly due to very less false posi
 tives and concurrency is also improved.<strong>NOTE:</strong> when BATCH_SORTis configured, it is recommended to keep <em><strong>carbon.load.batch.sort.size.inmb</strong></em> &gt; <em><strong>carbon.blockletgroup.size.in.mb</strong></em>
+</td>
 </tr>
 <tr>
-<td>carbon.sort.intermediate.files.limit</td>
-<td>20</td>
-<td>Minimum number of intermediate files after which merged sort can be started (minValue = 2, maxValue=50).</td>
+<td>carbon.load.batch.sort.size.inmb</td>
+<td>0</td>
+<td>When  <em><strong>carbon.load.sort.scope</strong></em> is configured as <em><strong>BATCH_SORT</strong></em>,This configuration needs to be added to specify the batch size for sorting and writing to carbondata files.<strong>NOTE:</strong> It is recommended to keep the value around 45% of <em><strong>carbon.sort.storage.inmemory.size.inmb</strong></em> to avoid spill to disk.Also it is recommended to keep the value higher than <em><strong>carbon.blockletgroup.size.in.mb</strong></em>. Refer to <em>carbon.load.sort.scope</em> for more information on sort options and the advantages/disadvantges of each option.</td>
 </tr>
 <tr>
-<td>carbon.block.meta.size.reserved.percentage</td>
-<td>10</td>
-<td>Space reserved in percentage for writing block meta data in CarbonData file.</td>
+<td>carbon.dictionary.server.port</td>
+<td>2030</td>
+<td>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.Single pass loading can be enabled using the option <em><strong>carbon.options.single.pass</strong></em>.When this option is specified, a dictionary server will be internally started to handle the dictionary generation and query requests.This configuration specifies the port on which the server need to listen for incoming requests.Port value ranges between 0-65535</td>
 </tr>
 <tr>
-<td>carbon.csv.read.buffersize.byte</td>
-<td>1048576</td>
-<td>csv reading buffer size.</td>
+<td>carbon.merge.sort.prefetch</td>
+<td>true</td>
+<td>CarbonData writes every <em><strong>carbon.sort.size</strong></em> number of records to intermediate temp files during data loading to ensure memory footprint is within limits.These intermediate temp files will have to be sorted using merge sort before writing into CarbonData format.This configuration enables pre fetching of data from these temp files in order to optimize IO and speed up data loading process.</td>
 </tr>
 <tr>
-<td>carbon.merge.sort.reader.thread</td>
-<td>3</td>
-<td>Maximum no of threads used for reading intermediate files for final merging.</td>
+<td>carbon.loading.prefetch</td>
+<td>false</td>
+<td>CarbonData uses univocity parser to read csv files.This configuration is used to inform the parser whether it can prefetch the data from csv files to speed up the reading.<strong>NOTE:</strong> Enabling prefetch improves the data loading performance, but needs higher memory to keep more records which are read ahead from disk.</td>
 </tr>
 <tr>
-<td>carbon.concurrent.lock.retries</td>
-<td>100</td>
-<td>Specifies the maximum number of retries to obtain the lock for concurrent operations. This is used for concurrent loading.</td>
+<td>carbon.prefetch.buffersize</td>
+<td>1000</td>
+<td>When the configuration <em><strong>carbon.merge.sort.prefetch</strong></em> is configured to true, we need to set the number of records that can be prefetched.This configuration is used specify the number of records to be prefetched.**NOTE: **Configuring more number of records to be prefetched increases memory footprint as more records will have to be kept in memory.</td>
 </tr>
 <tr>
-<td>carbon.concurrent.lock.retry.timeout.sec</td>
-<td>1</td>
-<td>Specifies the interval between the retries to obtain the lock for concurrent operations.</td>
+<td>load_min_size_inmb</td>
+<td>256</td>
+<td>This configuration is used along with <em><strong>carbon.load.min.size.enabled</strong></em>.This determines the minimum size of input files to be considered for distribution among executors while data loading.<strong>NOTE:</strong> Refer to <em><strong>carbon.load.min.size.enabled</strong></em> for understanding when this configuration needs to be used and its advantages and disadvantages.</td>
 </tr>
 <tr>
-<td>carbon.lock.retries</td>
-<td>3</td>
-<td>Specifies the maximum number of retries to obtain the lock for any operations other than load.</td>
+<td>carbon.load.sortmemory.spill.percentage</td>
+<td>0</td>
+<td>During data loading, some data pages are kept in memory upto memory configured in <em><strong>carbon.sort.storage.inmemory.size.inmb</strong></em> beyond which they are spilled to disk as intermediate temporary sort files.This configuration determines after what percentage data needs to be spilled to disk.<strong>NOTE:</strong> Without this configuration, when the data pages occupy upto configured memory, new data pages would be dumped to disk and old pages are still maintained in disk.</td>
 </tr>
 <tr>
-<td>carbon.lock.retry.timeout.sec</td>
-<td>5</td>
-<td>Specifies the interval between the retries to obtain the lock for any operation other than load.</td>
+<td>carbon.load.directWriteHdfs.enabled</td>
+<td>false</td>
+<td>During data load all the carbondata files are written to local disk and finally copied to the target location in HDFS.Enabling this parameter will make carrbondata files to be written directly onto target HDFS location bypassing the local disk.<strong>NOTE:</strong> Writing directly to HDFS saves local disk IO(once for writing the files and again for copying to HDFS) there by improving the performance.But the drawback is when data loading fails or the application crashes, unwanted carbondata files will remain in the target HDFS location until it is cleared during next data load or by running <em>CLEAN FILES</em> DDL command</td>
 </tr>
 <tr>
-<td>carbon.skip.empty.line</td>
-<td>false</td>
-<td>Setting this property ignores the empty lines in the CSV file during the data load</td>
+<td>carbon.options.serialization.null.format</td>
+<td>\N</td>
+<td>Based on the business scenarios, some columns might need to be loaded with null values.As null value cannot be written in csv files, some special characters might be adopted to specify null values.This configuration can be used to specify the null values format in the data being loaded.</td>
 </tr>
 <tr>
-<td>carbon.enable.calculate.size</td>
-<td>true</td>
-<td>
-<strong>For Load Operation</strong>: Setting this property calculates the size of the carbon data file (.carbondata) and carbon index file (.carbonindex) for every load and updates the table status file. <strong>For Describe Formatted</strong>: Setting this property calculates the total size of the carbon data files and carbon index files for the respective table and displays in describe formatted command.</td>
+<td>carbon.sort.storage.inmemory.size.inmb</td>
+<td>512</td>
+<td>CarbonData writes every <em><strong>carbon.sort.size</strong></em> number of records to intermediate temp files during data loading to ensure memory footprint is within limits.When <em><strong>enable.unsafe.sort</strong></em> configuration is enabled, instead of using <em><strong>carbon.sort.size</strong></em> which is based on rows count, size occupied in memory is used to determine when to flush data pages to intermediate temp files.This configuration determines the memory to be used for storing data pages in memory.<strong>NOTE:</strong> Configuring a higher values ensures more data is maintained in memory and hence increases data loading performance due to reduced or no IO.Based on the memory availability in the nodes of the cluster, configure the values accordingly.</td>
 </tr>
 </tbody>
 </table>
-<ul>
-<li><strong>Compaction Configuration</strong></li>
-</ul>
+<h2>
+<a id="compaction-configuration" class="anchor" href="#compaction-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Compaction Configuration</h2>
 <table>
 <thead>
 <tr>
@@ -599,30 +542,80 @@
 </thead>
 <tbody>
 <tr>
+<td>carbon.number.of.cores.while.compacting</td>
+<td>2</td>
+<td>Number of cores to be used while compacting data.This also determines the number of threads to be used to read carbondata files in parallel.</td>
+</tr>
+<tr>
+<td>carbon.compaction.level.threshold</td>
+<td>4, 3</td>
+<td>Each CarbonData load will create one segment, if every load is small in size it will generate many small file over a period of time impacting the query performance.This configuration is for minor compaction which decides how many segments to be merged. Configuration is of the form (x,y). Compaction will be triggered for every x segments and form a single level 1 compacted segment.When the number of compacted level 1 segments reach y, compaction will be triggered again to merge them to form a single level 2 segment. For example: If it is set as 2, 3 then minor compaction will be triggered for every 2 segments. 3 is the number of level 1 compacted segments which is further compacted to new segment.<strong>NOTE:</strong> When <em><strong>carbon.enable.auto.load.merge</strong></em> is <strong>true</strong>, Configuring higher values cause overall data loading time to increase as compaction will be triggered after data loading is complete but status is not returned till compaction is
  complete. But compacting more number of segments can increase query performance.Hence optimal values needs to be configured based on the business scenario.Valid values are bwteen 0 to 100.</td>
+</tr>
+<tr>
+<td>carbon.major.compaction.size</td>
+<td>1024</td>
+<td>To improve query performance and All the segments can be merged and compacted to a single segment upto configured size.This Major compaction size can be configured using this parameter. Sum of the segments which is below this threshold will be merged. This value is expressed in MB.</td>
+</tr>
+<tr>
+<td>carbon.horizontal.compaction.enable</td>
+<td>true</td>
+<td>CarbonData supports DELETE/UPDATE functionality by creating delta data files for existing carbondata files.These delta files would grow as more number of DELETE/UPDATE operations are performed.Compaction of these delta files are termed as horizontal compaction.This configuration is used to turn ON/OFF horizontal compaction. After every DELETE and UPDATE statement, horizontal compaction may occur in case the delta (DELETE/ UPDATE) files becomes more than specified threshold.**NOTE: **Having many delta files will reduce the query performance as scan has to happen on all these files before the final state of data can be decided.Hence it is advisable to keep horizontal compaction enabled and configure reasonable values to <em><strong>carbon.horizontal.UPDATE.compaction.threshold</strong></em> and <em><strong>carbon.horizontal.DELETE.compaction.threshold</strong></em>
+</td>
+</tr>
+<tr>
+<td>carbon.horizontal.update.compaction.threshold</td>
+<td>1</td>
+<td>This configuration specifies the threshold limit on number of UPDATE delta files within a segment. In case the number of delta files goes beyond the threshold, the UPDATE delta files within the segment becomes eligible for horizontal compaction and are compacted into single UPDATE delta file.Values range between 1 to 10000.</td>
+</tr>
+<tr>
+<td>carbon.horizontal.delete.compaction.threshold</td>
+<td>1</td>
+<td>This configuration specifies the threshold limit on number of DELETE delta files within a block of a segment. In case the number of delta files goes beyond the threshold, the DELETE delta files for the particular block of the segment becomes eligible for horizontal compaction and are compacted into single DELETE delta file.Values range between 1 to 10000.</td>
+</tr>
+<tr>
+<td>carbon.update.segment.parallelism</td>
+<td>1</td>
+<td>CarbonData processes the UPDATE operations by grouping records belonging to a segment into a single executor task.When the amount of data to be updated is more, this behavior causes problems like restarting of executor due to low memory and data-spill related errors.This property specifies the parallelism for each segment during update.<strong>NOTE:</strong> It is recommended to set this value to a multiple of the number of executors for balance.Values range between 1 to 1000.</td>
+</tr>
+<tr>
 <td>carbon.numberof.preserve.segments</td>
 <td>0</td>
-<td>If the user wants to preserve some number of segments from being compacted then he can set this property. Example: carbon.numberof.preserve.segments = 2 then 2 latest segments will always be excluded from the compaction. No segments will be preserved by default.</td>
+<td>If the user wants to preserve some number of segments from being compacted then he can set this configuration. Example: carbon.numberof.preserve.segments = 2 then 2 latest segments will always be excluded from the compaction. No segments will be preserved by default.<strong>NOTE:</strong> This configuration is useful when the chances of input data can be wrong due to environment scenarios.Preserving some of the latest segments from being compacted can help to easily delete the wrongly loaded segments.Once compacted,it becomes more difficult to determine the exact data to be deleted(except when data is incrementing according to time)</td>
 </tr>
 <tr>
 <td>carbon.allowed.compaction.days</td>
 <td>0</td>
-<td>Compaction will merge the segments which are loaded with in the specific number of days configured. Example: If the configuration is 2, then the segments which are loaded in the time frame of 2 days only will get merged. Segments which are loaded 2 days apart will not be merged. This is disabled by default.</td>
+<td>This configuration is used to control on the number of recent segments that needs to be compacted, ignoring the older ones.This congifuration is in days.For Example: If the configuration is 2, then the segments which are loaded in the time frame of past 2 days only will get merged. Segments which are loaded earlier than 2 days will not be merged. This configuration is disabled by default.<strong>NOTE:</strong> This configuration is useful when a bulk of history data is loaded into the carbondata.Query on this data is less frequent.In such cases involving these segments also into compacation will affect the resource consumption, increases overall compaction time.</td>
 </tr>
 <tr>
 <td>carbon.enable.auto.load.merge</td>
 <td>false</td>
-<td>To enable compaction while data loading.</td>
+<td>Compaction can be automatically triggered once data load completes.This ensures that the segments are merged in time and thus query times doesnt increase with increase in segments.This configuration enables to do compaction along with data loading.**NOTE: **Compaction will be triggered once the data load completes.But the status of data load wait till the compaction is completed.Hence it might look like data loading time has increased, but thats not the case.Moreover failure of compaction will not affect the data loading status.If data load had completed successfully, the status would be updated and segments are committed.However, failure while data loading, will not trigger compaction and error is returned immediately.</td>
 </tr>
 <tr>
 <td>carbon.enable.page.level.reader.in.compaction</td>
 <td>true</td>
-<td>Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory.</td>
+<td>Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory.<strong>NOTE:</strong> Please refer to <a href="./file-structure-of-carbondata.html">file-structure-of-carbondata</a> to understand the storage format of CarbonData and concepts of pages.</td>
+</tr>
+<tr>
+<td>carbon.concurrent.compaction</td>
+<td>true</td>
+<td>Compaction of different tables can be executed concurrently.This configuration determines whether to compact all qualifying tables in parallel or not.**NOTE: **Compacting concurrently is a resource demanding operation and needs more resouces there by affecting the query performance also.This configuration is <strong>deprecated</strong> and might be removed in future releases.</td>
+</tr>
+<tr>
+<td>carbon.compaction.prefetch.enable</td>
+<td>false</td>
+<td>Compaction operation is similar to Query + data load where in data from qualifying segments are queried and data loading performed to generate a new single segment.This configuration determines whether to query ahead data from segments and feed it for data loading.**NOTE: **This configuration is disabled by default as it needs extra resources for querying ahead extra data.Based on the memory availability on the cluster, user can enable it to improve compaction performance.</td>
+</tr>
+<tr>
+<td>carbon.merge.index.in.segment</td>
+<td>true</td>
+<td>Each CarbonData file has a companion CarbonIndex file which maintains the metadata about the data.These CarbonIndex files are read and loaded into driver and is used subsequently for pruning of data during queries.These CarbonIndex files are very small in size(few KB) and are many.Reading many small files from HDFS is not efficient and leads to slow IO performance.Hence these CarbonIndex files belonging to a segment can be combined into  a single file and read once there by increasing the IO throughput.This configuration enables to merge all the CarbonIndex files into a single MergeIndex file upon data loading completion.<strong>NOTE:</strong> Reading a single big file is more efficient in HDFS and IO throughput is very high.Due to this the time needed to load the index files into memory when query is received for the first time on that table is significantly reduced and there by significantly reduces the delay in serving the first query.</td>
 </tr>
 </tbody>
 </table>
-<ul>
-<li><strong>Query Configuration</strong></li>
-</ul>
+<h2>
+<a id="query-configuration" class="anchor" href="#query-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Query Configuration</h2>
 <table>
 <thead>
 <tr>
@@ -633,6 +626,16 @@
 </thead>
 <tbody>
 <tr>
+<td>carbon.max.driver.lru.cache.size</td>
+<td>-1</td>
+<td>Maximum memory <strong>(in MB)</strong> upto which the driver process can cache the data (BTree and dictionary values). Beyond this, least recently used data will be removed from cache before loading new set of values.Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.<strong>NOTE:</strong> Minimum number of entries that needs to be removed from cache in order to load the new set of data is determined and unloaded.ie.,for example if 3 cache entries qualify for pre-emption, out of these, those entries that free up more cache memory is removed prior to others.</td>
+</tr>
+<tr>
+<td>carbon.max.executor.lru.cache.size</td>
+<td>-1</td>
+<td>Maximum memory <strong>(in MB)</strong> upto which the executor process can cache the data (BTree and reverse dictionary values).Default value of -1 means there is no memory limit for caching. Only integer values greater than 0 are accepted.<strong>NOTE:</strong> If this parameter is not configured, then the value of <em><strong>carbon.max.driver.lru.cache.size</strong></em> will be used.</td>
+</tr>
+<tr>
 <td>max.query.execution.time</td>
 <td>60</td>
 <td>Maximum time allowed for one query to be executed. The value is in minutes.</td>
@@ -640,12 +643,12 @@
 <tr>
 <td>carbon.enableMinMax</td>
 <td>true</td>
-<td>Min max is feature added to enhance query performance. To disable this feature, set it false.</td>
+<td>CarbonData maintains the metadata which enables to prune unnecessary files from being scanned as per the query conditions.To achieve pruning, Min,Max of each column is maintined.Based on the filter condition in the query, certain data can be skipped from scanning by matching the filter value against the min,max values of the column(s) present in that carbondata file.This pruing enhances query performance significantly.</td>
 </tr>
 <tr>
 <td>carbon.dynamicallocation.schedulertimeout</td>
 <td>5</td>
-<td>Specifies the maximum time (unit in seconds) the scheduler can wait for executor to be active. Minimum value is 5 sec and maximum value is 15 sec.</td>
+<td>CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData.To determine the number of tasks that can be scheduled, knowing the count of active executors is necessary.When dynamic allocation is enabled on a YARN based spark cluster,execuor processes are shutdown if no request is received for a particular amount of time.The executors are brought up when the requet is received again.This configuration specifies the maximum time (unit in seconds) the carbon scheduler can wait for executor to be active. Minimum value is 5 sec and maximum value is 15 sec.**NOTE: **Waiting for longer time leads to slow query response time.Moreover it might be possible that YARN is not able to start the executors and waiting is not beneficial.</td>
 </tr>
 <tr>
 <td>carbon.scheduler.minregisteredresourcesratio</td>
@@ -657,35 +660,76 @@
 <td>false</td>
 <td>If set to true, it will use CarbonReader to do distributed scan directly instead of using compute framework like spark, thus avoiding limitation of compute framework like SQL optimizer and task scheduling overhead.</td>
 </tr>
-</tbody>
-</table>
-<ul>
-<li><strong>Global Dictionary Configurations</strong></li>
-</ul>
-<table>
-<thead>
 <tr>
-<th>Parameter</th>
-<th>Default Value</th>
-<th>Description</th>
+<td>carbon.search.query.timeout</td>
+<td>10s</td>
+<td>Time within which the result is expected from the workers;beyond which the query is terminated</td>
 </tr>
-</thead>
-<tbody>
 <tr>
-<td>carbon.cutOffTimestamp</td>
-<td></td>
-<td>Sets the start date for calculating the timestamp. Java counts the number of milliseconds from start of "1970-01-01 00:00:00". This property is used to customize the start of position. For example "2000-01-01 00:00:00". The date must be in the form "carbon.timestamp.format".</td>
+<td>carbon.search.scan.thread</td>
+<td>num of cores available in worker node</td>
+<td>Number of cores to be used in each worker for performing scan.</td>
 </tr>
 <tr>
-<td>carbon.timegranularity</td>
-<td>SECOND</td>
-<td>The property used to set the data granularity level DAY, HOUR, MINUTE, or SECOND.</td>
+<td>carbon.search.master.port</td>
+<td>10020</td>
+<td>Port on which the search master listens for incoming query requests</td>
+</tr>
+<tr>
+<td>carbon.search.worker.port</td>
+<td>10021</td>
+<td>Port on which search master communicates with the workers.</td>
+</tr>
+<tr>
+<td>carbon.search.worker.workload.limit</td>
+<td>10 * <em>carbon.search.scan.thread</em>
+</td>
+<td>Maximum number of active requests that can be sent to a worker.Beyond which the request needs to be rescheduled for later time or to a different worker.</td>
+</tr>
+<tr>
+<td>carbon.detail.batch.size</td>
+<td>100</td>
+<td>The buffer size to store records, returned from the block scan. In limit scenario this parameter is very important. For example your query limit is 1000. But if we set this value to 3000 that means we get 3000 records from scan but spark will only take 1000 rows. So the 2000 remaining are useless. In one Finance test case after we set it to 100, in the limit 1000 scenario the performance increase about 2 times in comparison to if we set this value to 12000.</td>
+</tr>
+<tr>
+<td>carbon.enable.vector.reader</td>
+<td>true</td>
+<td>Spark added vector processing to optimize cpu cache miss and there by increase the query performance.This configuration enables to fetch data as columnar batch of size 4*1024 rows instead of fetching data row by row and provide it to spark so that there is improvement in  select queries performance.</td>
+</tr>
+<tr>
+<td>carbon.task.distribution</td>
+<td>block</td>
+<td>CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData.Each of these task distribution suggestions has its own advantages and disadvantages.Based on the customer use case, appropriate task distribution can be configured.<strong>block</strong>: Setting this value will launch one task per block. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. <strong>custom</strong>: Setting this value will group the blocks and distribute it uniformly to the available resources in the cluster. This enhances the query performance but not suggested in case of concurrent queries and queries having big shuffling scenarios. <strong>blocklet</strong>: Setting this value will launch one task per blocklet. This setting is suggested in case of concurrent queries and queries having big shuffling scenarios. <strong>merge_smal
 l_files</strong>: Setting this value will merge all the small carbondata files upto a bigger size configured by <em><strong>spark.sql.files.maxPartitionBytes</strong></em> (128 MB is the default value,it is configurable) during querying. The small carbondata files are combined to a map task to reduce the number of read task. This enhances the performance.</td>
+</tr>
+<tr>
+<td>carbon.custom.block.distribution</td>
+<td>false</td>
+<td>CarbonData has its own scheduling algorithm to suggest to Spark on how many tasks needs to be launched and how much work each task need to do in a Spark cluster for any query on CarbonData.When this configuration is true, CarbonData would distribute the available blocks to be scanned among the available number of cores.For Example:If there are 10 blocks to be scanned and only 3 tasks can be run(only 3 executor cores available in the cluster), CarbonData would combine blocks as 4,3,3 and give it to 3 tasks to run.<strong>NOTE:</strong> When this configuration is false, as per the <em><strong>carbon.task.distribution</strong></em> configuration, each block/blocklet would be given to each task.</td>
+</tr>
+<tr>
+<td>enable.query.statistics</td>
+<td>false</td>
+<td>CarbonData has extensive logging which would be useful for debugging issues related to performance or hard to locate issues.This configuration when made <em><strong>true</strong></em> would log additional query statistics information to more accurately locate the issues being debugged.<strong>NOTE:</strong> Enabling this would log more debug information to log files, there by increasing the log files size significantly in short span of time.It is advised to configure the log files size, retention of log files parameters in log4j properties appropriately.Also extensive logging is an increased IO operation and hence over all query performance might get reduced.Therefore it is recommened to enable this configuration only for the duration of debugging.</td>
+</tr>
+<tr>
+<td>enable.unsafe.in.query.processing</td>
+<td>true</td>
+<td>CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations.This configuration enables to use unsafe functions in CarbonData while scanning the  data during query.</td>
+</tr>
+<tr>
+<td>carbon.query.validate.directqueryondatamap</td>
+<td>true</td>
+<td>CarbonData supports creating pre-aggregate table datamaps as an independent tables.For some debugging purposes, it might be required to directly query from such datamap tables.This configuration allows to query on such datamaps.</td>
+</tr>
+<tr>
+<td>carbon.heap.memory.pooling.threshold.bytes</td>
+<td>1048576</td>
+<td>CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations.Using unsafe, memory can be allocated on Java Heap or off heap.This configuration controlls the allocation mechanism on Java HEAP.If the heap memory allocations of the given size is greater or equal than this value,it should go through the pooling mechanism.But if set this size to -1, it should not go through the pooling mechanism.Default value is 1048576(1MB, the same as Spark).Value to be specified in bytes.</td>
 </tr>
 </tbody>
 </table>
 <h2>
-<a id="spark-configuration" class="anchor" href="#spark-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Spark Configuration</h2>
-<p><b></b></p><p align="center">Spark Configuration Reference in spark-defaults.conf</p>
+<a id="data-mutation-configuration" class="anchor" href="#data-mutation-configuration" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Mutation Configuration</h2>
 <table>
 <thead>
 <tr>
@@ -696,14 +740,24 @@
 </thead>
 <tbody>
 <tr>
-<td>spark.driver.memory</td>
-<td>1g</td>
-<td>Amount of memory to be used by the driver process.</td>
+<td>carbon.insert.persist.enable</td>
+<td>false</td>
+<td>CarbonData does loading in 2 major steps.1st step reads from the input source and generates the dictionary values.2nd step reads from the source again and encodes the data with the dictionary values, perform index calculations and writes in CarbonData format. Suppose we are loading the CarbonData table using another table as source(using insert into) and the source table is being loaded in parallel, there can be cases where some data got inserted into the source table after CarbonData generated for the target table in which case some new records which does not have dictionary values generated gets read leading to inconsistency. To avoid this condition we can persist the dataset of RDD/dataframe into MEMORY_AND_DISK(default value) and perform insert into operation. This ensures the data read from source table is cached and is not read again from the source there by ensuring consistency between dictionary generation and writing to CarbonData format steps. By default this value is 
 false as concurrent loading into source table is not the scenario majority of the times.<strong>NOTE:</strong> This configuration can reduce the insert into execution time as data need not be re read; but increases the memory foot print.</td>
 </tr>
 <tr>
-<td>spark.executor.memory</td>
-<td>1g</td>
-<td>Amount of memory to be used per executor process.</td>
+<td>carbon.insert.storage.level</td>
+<td>MEMORY_AND_DISK</td>
+<td>Storage level to persist dataset of a RDD/dataframe.Applicable when <em><strong>carbon.insert.persist.enable</strong></em> is <strong>true</strong>, if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. <a href="http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence" rel="nofollow">See detail</a>.</td>
+</tr>
+<tr>
+<td>carbon.update.persist.enable</td>
+<td>true</td>
+<td>Configuration to enable the dataset of RDD/dataframe to persist data. Enabling this will reduce the execution time of UPDATE operation.</td>
+</tr>
+<tr>
+<td>carbon.update.storage.level</td>
+<td>MEMORY_AND_DISK</td>
+<td>Storage level to persist dataset of a RDD/dataframe.Applicable when <em><strong>carbon.update.persist.enable</strong></em> is <strong>true</strong>, if user's executor has less memory, set this parameter to 'MEMORY_AND_DISK_SER' or other storage level to correspond to different environment. <a href="http://spark.apache.org/docs/latest/rdd-programming-guide.html#rdd-persistence" rel="nofollow">See detail</a>.</td>
 </tr>
 </tbody>
 </table>
@@ -767,6 +821,10 @@
 <tbody>
 <tr>
 <td>carbon.options.bad.records.logger.enable</td>
+<td>CarbonData can identify the records that are not conformant to schema and isolate them as bad records.Enabling this configuration will make CarbonData to log such bad records.<strong>NOTE:</strong> If the input data contains many bad records, logging them will slow down the over all data loading throughput.The data load operation status would depend on the configuration in <em><strong>carbon.bad.records.action</strong></em>.</td>
+</tr>
+<tr>
+<td>carbon.options.bad.records.logger.enable</td>
 <td>To enable or disable bad record logger.</td>
 </tr>
 <tr>
@@ -783,7 +841,7 @@
 </tr>
 <tr>
 <td>carbon.options.single.pass</td>
-<td>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.</td>
+<td>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.<strong>NOTE:</strong> Enabling this starts a new dictionary server to handle dictionary generation requests during data loading.Without this option, the input csv files will have to read twice.Once while dictionary generation and persisting to the dictionary files.second when the data loading need to convert the input data into carbondata format.Enabling this optimizes the optimizes to read the input data only once there by reducing IO and hence over all data loading time.If concurrent data loading needs to be supported, consider tuning <em><strong>dictionary.worker.threads</strong></em>.Port on which the dictionary server
  need to listen on can be configured using the configuration <em><strong>carbon.dictionary.server.port</strong></em>.</td>
 </tr>
 <tr>
 <td>carbon.options.bad.record.path</td>
@@ -791,12 +849,36 @@
 </tr>
 <tr>
 <td>carbon.custom.block.distribution</td>
-<td>Specifies whether to use the Spark or Carbon block distribution feature.</td>
+<td>Specifies whether to use the Spark or Carbon block distribution feature.**NOTE: **Refer to <a href="#query-configuration">Query Configuration</a>#carbon.custom.block.distribution for more details on CarbonData scheduler.</td>
 </tr>
 <tr>
 <td>enable.unsafe.sort</td>
 <td>Specifies whether to use unsafe sort during data loading. Unsafe sort reduces the garbage collection during data load operation, resulting in better performance.</td>
 </tr>
+<tr>
+<td>carbon.options.dateformat</td>
+<td>Specifies the data format of the date columns in the data being loaded</td>
+</tr>
+<tr>
+<td>carbon.options.timestampformat</td>
+<td>Specifies the timestamp format of the time stamp columns in the data being loaded</td>
+</tr>
+<tr>
+<td>carbon.options.sort.scope</td>
+<td>Specifies how the current data load should be sorted with.**NOTE: **Refer to <a href="#data-loading-configuration">Data Loading Configuration</a>#carbon.sort.scope for detailed information.</td>
+</tr>
+<tr>
+<td>carbon.options.global.sort.partitions</td>
+<td></td>
+</tr>
+<tr>
+<td>carbon.options.serialization.null.format</td>
+<td>Default Null value representation in the data being loaded.<strong>NOTE:</strong> Refer to <a href="#data-loading-configuration">Data Loading Configuration</a>#carbon.options.serialization.null.format for detailed information.</td>
+</tr>
+<tr>
+<td>carbon.query.directQueryOnDataMap.enabled</td>
+<td>Specifies whether datamap can be queried directly.This is useful for debugging purposes.**NOTE: **Refer to <a href="#query-configuration">Query Configuration</a>#carbon.query.validate.directqueryondatamap for detailed information.</td>
+</tr>
 </tbody>
 </table>
 <p><strong>Examples:</strong></p>
@@ -824,6 +906,17 @@
 <p>Failure will be displayed in the UI.</p>
 </li>
 </ul>
+<script>
+$(function() {
+  // Show selected style on nav item
+  $('.b-nav__docs').addClass('selected');
+
+  // Display docs subnav items
+  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
+    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
+  }
+});
+</script>
 </div>
 </div>
 </div>
@@ -839,4 +932,4 @@
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/css/style.css
----------------------------------------------------------------------
diff --git a/content/css/style.css b/content/css/style.css
index 94b8fbf..88fd05f 100644
--- a/content/css/style.css
+++ b/content/css/style.css
@@ -4,7 +4,7 @@
 
 *{margin: 0; padding: 0;}
 
-body{background:#fdfdfd; color:#000 ; font-family:'Open Sans', arial, helvatica,  sans-serif; font-size:14px; line-height:22px;}
+body{background:#fdfdfd; color:#000 ; font-family:'Helvetica Neue', arial, helvatica,  sans-serif; font-size:14px; line-height:22px;}
 
 a{-webkit-transition: all .1s ease-in-out;
    -moz-transition: all .1s ease-in-out;
@@ -1301,3 +1301,77 @@ box-shadow: 0 3px 2px #aaa }
     white-space: nowrap;
 }
 
+.mdcontent {
+float:right;
+width:80%;
+padding-left:30px;
+}
+
+.verticalnavbar {
+    float: left;
+    text-transform: uppercase;
+    width: 15%;
+    font-family:"Helvetica Neue";
+    padding-top: 90px;
+    position: fixed;
+}
+.nav__item,
+.nav__item__with__subs {
+    color: #000000;
+    border-right: 2px solid #000000;
+    display: block;
+    padding-top: 1.5rem;
+    position: relative;
+}
+.nav__item__with__subs {
+    padding-top: 0;
+}
+.nav__sub__anchor,
+.nav__sub__item {
+    border-right: none;
+}
+.nav__sub__item {
+    display: none;
+    color: #888888;
+    font-size: 1.2rem;
+    text-transform: capitalize;
+}
+.nav__item__with__subs--expanded .nav__sub__item {
+    display: block;
+}
+.nav__item:first-of-type {
+    padding-top: 0;
+}
+.nav__item__with__subs .nav__item:first-of-type {
+    padding-top: 1.5rem;
+}
+.nav__item::after {
+    content: "";
+    display: block;
+    height: 2.1rem;
+    width: 1.1rem;
+    border-radius: 1rem;
+    -moz-border-radius: 1rem;
+    -webkit-border-radius: 1rem;
+    border: 2px solid #000000;
+    background: #FFFFFF;
+    position: absolute;
+    right: -.7rem;
+    top: 1.7rem;
+    opacity: 0;
+    transition: opacity .2s ease-out;
+}
+.nav__item.selected::after {
+    opacity: 1;
+}
+.nav__item.selected:first-of-type::after {
+    top: .4rem;
+}
+.nav__item__with__subs .nav__item:first-of-type::after {
+    top: 1.7rem;
+}
+verticalnavbar .btn {
+    display: block;
+    margin-top: 4rem;
+}
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/data-management-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/data-management-on-carbondata.html b/content/data-management-on-carbondata.html
index 566bb8e..bb5ae78 100644
--- a/content/data-management-on-carbondata.html
+++ b/content/data-management-on-carbondata.html
@@ -1318,4 +1318,4 @@ Future {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/44eed099/content/data-management.html
----------------------------------------------------------------------
diff --git a/content/data-management.html b/content/data-management.html
index 93528d8..a9086ca 100644
--- a/content/data-management.html
+++ b/content/data-management.html
@@ -410,4 +410,4 @@ on the basis of column expression and optional filter conditions.</p>
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
\ No newline at end of file
+</html>


[21/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/bloomfilter-datamap-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/bloomfilter-datamap-guide.md b/src/site/markdown/bloomfilter-datamap-guide.md
index dd590e1..b2e7d60 100644
--- a/src/site/markdown/bloomfilter-datamap-guide.md
+++ b/src/site/markdown/bloomfilter-datamap-guide.md
@@ -73,7 +73,7 @@ For instance, main table called **datamap_test** which is defined as:
     age int,
     city string,
     country string)
-  STORED BY 'carbondata'
+  STORED AS carbondata
   TBLPROPERTIES('SORT_COLUMNS'='id')
   ```
 
@@ -146,14 +146,3 @@ You can refer to the corresponding section in `CarbonData Lucene DataMap`.
  there is still a chance that BloomFilter datamap can enhance the performance for concurrent query.
 + Note that BloomFilter datamap will decrease the data loading performance and may cause slightly storage expansion (for datamap index file).
 
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__datamap').addClass('selected');
-  
-  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
-    // Display datamap subnav items
-    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/configuration-parameters.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/configuration-parameters.md b/src/site/markdown/configuration-parameters.md
index de72439..c8c74f2 100644
--- a/src/site/markdown/configuration-parameters.md
+++ b/src/site/markdown/configuration-parameters.md
@@ -16,7 +16,7 @@
 -->
 
 # Configuring CarbonData
- This guide explains the configurations that can be used to tune CarbonData to achieve better performance.Some of the properties can be set dynamically and are explained in the section Dynamic Configuration In CarbonData Using SET-RESET.Most of the properties that control the internal settings have reasonable default values.They are listed along with the properties along with explanation.
+ This guide explains the configurations that can be used to tune CarbonData to achieve better performance.Most of the properties that control the internal settings have reasonable default values.They are listed along with the properties along with explanation.
 
  * [System Configuration](#system-configuration)
  * [Data Loading Configuration](#data-loading-configuration)
@@ -59,7 +59,7 @@ This section provides the details of all the configurations required for the Car
 | carbon.bad.records.action | FAIL | CarbonData in addition to identifying the bad records, can take certain actions on such data.This configuration can have four types of actions for bad records namely FORCE, REDIRECT, IGNORE and FAIL. If set to FORCE then it auto-corrects the data by storing the bad records as NULL. If set to REDIRECT then bad records are written to the raw CSV instead of being loaded. If set to IGNORE then bad records are neither loaded nor written to the raw CSV. If set to FAIL then data loading fails if any bad records are found. |
 | carbon.options.is.empty.data.bad.record | false | Based on the business scenarios, empty("" or '' or ,,) data can be valid or invalid. This configuration controls how empty data should be treated by CarbonData. If false, then empty ("" or '' or ,,) data will not be considered as bad record and vice versa. |
 | carbon.options.bad.record.path | (none) | Specifies the HDFS path where bad records are to be stored. By default the value is Null. This path must to be configured by the user if ***carbon.options.bad.records.logger.enable*** is **true** or ***carbon.bad.records.action*** is **REDIRECT**. |
-| carbon.blockletgroup.size.in.mb | 64 | Please refer to [file-structure-of-carbondata](./file-structure-of-carbondata.md ) to understand the storage format of CarbonData.The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of each blocklet group. Higher value results in better sequential IO access.The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB).**NOTE:** Configuring a higher value might lead to poor performance as an entire blocklet group will have to read into memory before processing.For filter queries with limit, it is **not advisable** to have a bigger blocklet size.For Aggregation queries which need to return more number of rows,bigger blocklet size is advisable. |
+| carbon.blockletgroup.size.in.mb | 64 | Please refer to [file-structure-of-carbondata](./file-structure-of-carbondata.md#carbondata-file-format) to understand the storage format of CarbonData.The data are read as a group of blocklets which are called blocklet groups. This parameter specifies the size of each blocklet group. Higher value results in better sequential IO access.The minimum value is 16MB, any value lesser than 16MB will reset to the default value (64MB).**NOTE:** Configuring a higher value might lead to poor performance as an entire blocklet group will have to read into memory before processing.For filter queries with limit, it is **not advisable** to have a bigger blocklet size.For Aggregation queries which need to return more number of rows,bigger blocklet size is advisable. |
 | carbon.sort.file.write.buffer.size | 16384 | CarbonData sorts and writes data to intermediate files to limit the memory usage.This configuration determines the buffer size to be used for reading and writing such files. **NOTE:** This configuration is useful to tune IO and derive optimal performance.Based on the OS and underlying harddisk type, these values can significantly affect the overall performance.It is ideal to tune the buffersize equivalent to the IO buffer size of the OS.Recommended range is between 10240 to 10485760 bytes. |
 | carbon.sort.intermediate.files.limit | 20 | CarbonData sorts and writes data to intermediate files to limit the memory usage.Before writing the target carbondat file, the data in these intermediate files needs to be sorted again so as to ensure the entire data in the data load is sorted.This configuration determines the minimum number of intermediate files after which merged sort is applied on them sort the data.**NOTE:** Intermediate merging happens on a separate thread in the background.Number of threads used is determined by ***carbon.merge.sort.reader.thread***.Configuring a low value will cause more time to be spent in merging these intermediate merged files which can cause more IO.Configuring a high value would cause not to use the idle threads to do intermediate sort merges.Range of recommended values are between 2 and 50 |
 | carbon.csv.read.buffersize.byte | 1048576 | CarbonData uses Hadoop InputFormat to read the csv files.This configuration value is used to pass buffer size as input for the Hadoop MR job when reading the csv files.This value is configured in bytes.**NOTE:** Refer to ***org.apache.hadoop.mapreduce.InputFormat*** documentation for additional information. |
@@ -70,7 +70,7 @@ This section provides the details of all the configurations required for the Car
 | carbon.enable.calculate.size | true | **For Load Operation**: Setting this property calculates the size of the carbon data file (.carbondata) and carbon index file (.carbonindex) for every load and updates the table status file. **For Describe Formatted**: Setting this property calculates the total size of the carbon data files and carbon index files for the respective table and displays in describe formatted command.**NOTE:** This is useful to determine the overall size of the carbondata table and also get an idea of how the table is growing in order to take up other backup strategy decisions. |
 | carbon.cutOffTimestamp | (none) | CarbonData has capability to generate the Dictionary values for the timestamp columns from the data itself without the need to store the computed dictionary values. This configuration sets the start date for calculating the timestamp. Java counts the number of milliseconds from start of "1970-01-01 00:00:00". This property is used to customize the start of position. For example "2000-01-01 00:00:00". **NOTE:** The date must be in the form ***carbon.timestamp.format***. CarbonData supports storing data for upto 68 years.For example, if the cut-off time is 1970-01-01 05:30:00, then data upto 2038-01-01 05:30:00 will be supported by CarbonData. |
 | carbon.timegranularity | SECOND | The configuration is used to specify the data granularity level such as DAY, HOUR, MINUTE, or SECOND.This helps to store more than 68 years of data into CarbonData. |
-| carbon.use.local.dir | false | CarbonData during data loading, writes files to local temp directories before copying the files to HDFS.This configuration is used to specify whether CarbonData can write locally to tmp directory of the container or to the YARN application directory. |
+| carbon.use.local.dir | false | CarbonData,during data loading, writes files to local temp directories before copying the files to HDFS.This configuration is used to specify whether CarbonData can write locally to tmp directory of the container or to the YARN application directory. |
 | carbon.use.multiple.temp.dir | false | When multiple disks are present in the system, YARN is generally configured with multiple disks to be used as temp directories for managing the containers.This configuration specifies whether to use multiple YARN local directories during data loading for disk IO load balancing.Enable ***carbon.use.local.dir*** for this configuration to take effect.**NOTE:** Data Loading is an IO intensive operation whose performance can be limited by the disk IO threshold, particularly during multi table concurrent data load.Configuring this parameter, balances the disk IO across multiple disks there by improving the over all load performance. |
 | carbon.sort.temp.compressor | (none) | CarbonData writes every ***carbon.sort.size*** number of records to intermediate temp files during data loading to ensure memory footprint is within limits.These temporary files cab be compressed and written in order to save the storage space.This configuration specifies the name of compressor to be used to compress the intermediate sort temp files during sort procedure in data loading.The valid values are 'SNAPPY','GZIP','BZIP2','LZ4','ZSTD' and empty. By default, empty means that Carbondata will not compress the sort temp files.**NOTE:** Compressor will be useful if you encounter disk bottleneck.Since the data needs to be compressed and decompressed,it involves additional CPU cycles,but is compensated by the high IO throughput due to less data to be written or read from the disks. |
 | carbon.load.skewedDataOptimization.enabled | false | During data loading,CarbonData would divide the number of blocks equally so as to ensure all executors process same number of blocks.This mechanism satisfies most of the scenarios and ensures maximum parallel processing for optimal data loading performance.In some business scenarios, there might be scenarios where the size of blocks vary significantly and hence some executors would have to do more work if they get blocks containing more data. This configuration enables size based block allocation strategy for data loading.When loading, carbondata will use file size based block allocation strategy for task distribution. It will make sure that all the executors process the same size of data.**NOTE:** This configuration is useful if the size of your input data files varies widely, say 1MB~1GB.For this configuration to work effectively,knowing the data pattern and size is important and necessary. |
@@ -107,7 +107,7 @@ This section provides the details of all the configurations required for the Car
 | carbon.numberof.preserve.segments | 0 | If the user wants to preserve some number of segments from being compacted then he can set this configuration. Example: carbon.numberof.preserve.segments = 2 then 2 latest segments will always be excluded from the compaction. No segments will be preserved by default.**NOTE:** This configuration is useful when the chances of input data can be wrong due to environment scenarios.Preserving some of the latest segments from being compacted can help to easily delete the wrongly loaded segments.Once compacted,it becomes more difficult to determine the exact data to be deleted(except when data is incrementing according to time) |
 | carbon.allowed.compaction.days | 0 | This configuration is used to control on the number of recent segments that needs to be compacted, ignoring the older ones.This congifuration is in days.For Example: If the configuration is 2, then the segments which are loaded in the time frame of past 2 days only will get merged. Segments which are loaded earlier than 2 days will not be merged. This configuration is disabled by default.**NOTE:** This configuration is useful when a bulk of history data is loaded into the carbondata.Query on this data is less frequent.In such cases involving these segments also into compacation will affect the resource consumption, increases overall compaction time. |
 | carbon.enable.auto.load.merge | false | Compaction can be automatically triggered once data load completes.This ensures that the segments are merged in time and thus query times doesnt increase with increase in segments.This configuration enables to do compaction along with data loading.**NOTE: **Compaction will be triggered once the data load completes.But the status of data load wait till the compaction is completed.Hence it might look like data loading time has increased, but thats not the case.Moreover failure of compaction will not affect the data loading status.If data load had completed successfully, the status would be updated and segments are committed.However, failure while data loading, will not trigger compaction and error is returned immediately. |
-| carbon.enable.page.level.reader.in.compaction|true|Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory.**NOTE:** Please refer to [file-structure-of-carbondata](./file-structure-of-carbondata.md ) to understand the storage format of CarbonData and concepts of pages.|
+| carbon.enable.page.level.reader.in.compaction|true|Enabling page level reader for compaction reduces the memory usage while compacting more number of segments. It allows reading only page by page instead of reading whole blocklet to memory.**NOTE:** Please refer to [file-structure-of-carbondata](./file-structure-of-carbondata.md#carbondata-file-format) to understand the storage format of CarbonData and concepts of pages.|
 | carbon.concurrent.compaction | true | Compaction of different tables can be executed concurrently.This configuration determines whether to compact all qualifying tables in parallel or not.**NOTE: **Compacting concurrently is a resource demanding operation and needs more resouces there by affecting the query performance also.This configuration is **deprecated** and might be removed in future releases. |
 | carbon.compaction.prefetch.enable | false | Compaction operation is similar to Query + data load where in data from qualifying segments are queried and data loading performed to generate a new single segment.This configuration determines whether to query ahead data from segments and feed it for data loading.**NOTE: **This configuration is disabled by default as it needs extra resources for querying ahead extra data.Based on the memory availability on the cluster, user can enable it to improve compaction performance. |
 | carbon.merge.index.in.segment | true | Each CarbonData file has a companion CarbonIndex file which maintains the metadata about the data.These CarbonIndex files are read and loaded into driver and is used subsequently for pruning of data during queries.These CarbonIndex files are very small in size(few KB) and are many.Reading many small files from HDFS is not efficient and leads to slow IO performance.Hence these CarbonIndex files belonging to a segment can be combined into  a single file and read once there by increasing the IO throughput.This configuration enables to merge all the CarbonIndex files into a single MergeIndex file upon data loading completion.**NOTE:** Reading a single big file is more efficient in HDFS and IO throughput is very high.Due to this the time needed to load the index files into memory when query is received for the first time on that table is significantly reduced and there by significantly reduces the delay in serving the first query. |
@@ -235,16 +235,3 @@ RESET
 * Success will be recorded in the driver log.
 
 * Failure will be displayed in the UI.
-
-
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__docs').addClass('selected');
-
-  // Display docs subnav items
-  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
-    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/datamap-developer-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/datamap-developer-guide.md b/src/site/markdown/datamap-developer-guide.md
index 52c993c..6bac9b5 100644
--- a/src/site/markdown/datamap-developer-guide.md
+++ b/src/site/markdown/datamap-developer-guide.md
@@ -15,16 +15,5 @@ Currently, the provider string can be:
 
 When user issues `DROP DATAMAP dm ON TABLE main`, the corresponding DataMapProvider interface will be called.
 
-Details about [DataMap Management](./datamap-management.md#datamap-management) and supported [DSL](./datamap-management.md#overview) are documented [here](./datamap-management.md).
+Details about [DataMap Management](./datamap/datamap-management.md#datamap-management) and supported [DSL](./datamap/datamap-management.md#overview) are documented [here](./datamap/datamap-management.md).
 
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__docs').addClass('selected');
-
-  // Display docs subnav items
-  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
-    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/datamap-management.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/datamap-management.md b/src/site/markdown/datamap-management.md
index cc17d69..eee03a7 100644
--- a/src/site/markdown/datamap-management.md
+++ b/src/site/markdown/datamap-management.md
@@ -149,15 +149,3 @@ This feature applies for preaggregate datamap only
 Running Compaction command (`ALTER TABLE COMPACT`) on main table will **not automatically** compact the pre-aggregate tables created on the main table. User need to run Compaction command separately on each pre-aggregate table to compact them.
 
 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 tables. To further improve the query performance, compaction on pre-aggregate tables can be triggered to merge the segments and files in the pre-aggregate tables.
-
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__datamap').addClass('selected');
-  
-  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
-    // Display datamap subnav items
-    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/ddl-of-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/ddl-of-carbondata.md b/src/site/markdown/ddl-of-carbondata.md
index 5535a40..acaac43 100644
--- a/src/site/markdown/ddl-of-carbondata.md
+++ b/src/site/markdown/ddl-of-carbondata.md
@@ -75,15 +75,31 @@ CarbonData DDL statements are documented here,which includes:
   **NOTE:** CarbonData also supports "STORED AS carbondata" and "USING carbondata". Find example code at [CarbonSessionExample](https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala) in the CarbonData repo.
 ### Usage Guidelines
 
-**Supported properties:** [DICTIONARY_INCLUDE](#dictionary-encoding-configuration),[NO_INVERTED_INDEX](#inverted-index-configuration),[SORT_COLUMNS](#sort-columns-configuration),[SORT_SCOPE](#sort-scope-configuration),[TABLE_BLOCKSIZE](#table-block-size-configuration),[MAJOR_COMPACTION_SIZE](#table-compaction-configuration),
-
-[AUTO_LOAD_MERGE](#table-compaction-configuration),[COMPACTION_LEVEL_THRESHOLD](#table-compaction-configuration),[COMPACTION_PRESERVE_SEGMENTS](#table-compaction-configuration),[ALLOWED_COMPACTION_DAYS](#table-compaction-configuration),
-
-[streaming](#streaming),[LOCAL_DICTIONARY_ENABLE](#local-dictionary-configuration),[LOCAL_DICTIONARY_THRESHOLD](#local-dictionary-configuration),[LOCAL_DICTIONARY_INCLUDE](#local-dictionary-configuration),
-
-[LOCAL_DICTIONARY_EXCLUDE](#local-dictionary-configuration),[COLUMN_META_CACHE](#caching-minmax-value-for-required-columns),[CACHE_LEVEL](#caching-at-block-or-blocklet-level),[flat_folder](#support-flat-folder-same-as-hiveparquet),[LONG_STRING_COLUMNS](#string-longer-than-32000-characters),[BUCKETNUMBER](#bucketing),
-
-[BUCKETCOLUMNS](#bucketing)
+**Supported properties:**
+
+| Property                                                     | Description                                                  |
+| ------------------------------------------------------------ | ------------------------------------------------------------ |
+| [DICTIONARY_INCLUDE](#dictionary-encoding-configuration)     | Columns for which dictionary needs to be generated           |
+| [NO_INVERTED_INDEX](#inverted-index-configuration)           | Columns to exclude from inverted index generation            |
+| [SORT_COLUMNS](#sort-columns-configuration)                  | Columns to include in sort and its order of sort             |
+| [SORT_SCOPE](#sort-scope-configuration)                      | Sort scope of the load.Options include no sort, local sort ,batch sort and global sort |
+| [TABLE_BLOCKSIZE](#table-block-size-configuration)           | Size of blocks to write onto hdfs                            |
+| [MAJOR_COMPACTION_SIZE](#table-compaction-configuration)     | Size upto which the segments can be combined into one        |
+| [AUTO_LOAD_MERGE](#table-compaction-configuration)           | Whether to auto compact the segments                         |
+| [COMPACTION_LEVEL_THRESHOLD](#table-compaction-configuration) | Number of segments to compact into one segment               |
+| [COMPACTION_PRESERVE_SEGMENTS](#table-compaction-configuration) | Number of latest segments that needs to be excluded from compaction |
+| [ALLOWED_COMPACTION_DAYS](#table-compaction-configuration)   | Segments generated within the configured time limit in days will be compacted, skipping others |
+| [streaming](#streaming)                                      | Whether the table is a streaming table                       |
+| [LOCAL_DICTIONARY_ENABLE](#local-dictionary-configuration)   | Enable local dictionary generation                           |
+| [LOCAL_DICTIONARY_THRESHOLD](#local-dictionary-configuration) | Cardinality upto which the local dictionary can be generated |
+| [LOCAL_DICTIONARY_INCLUDE](#local-dictionary-configuration)  | Columns for which local dictionary needs to be generated.Useful when local dictionary need not be generated for all string/varchar/char columns |
+| [LOCAL_DICTIONARY_EXCLUDE](#local-dictionary-configuration)  | Columns for which local dictionary generation should be skipped.Useful when local dictionary need not be generated for few string/varchar/char columns |
+| [COLUMN_META_CACHE](#caching-minmax-value-for-required-columns) | Columns whose metadata can be cached in Driver for efficient pruning and improved query performance |
+| [CACHE_LEVEL](#caching-at-block-or-blocklet-level)           | Column metadata caching level.Whether to cache column metadata of block or blocklet |
+| [flat_folder](#support-flat-folder-same-as-hiveparquet)      | Whether to write all the carbondata files in a single folder.Not writing segments folder during incremental load |
+| [LONG_STRING_COLUMNS](#string-longer-than-32000-characters)  | Columns which are greater than 32K characters                |
+| [BUCKETNUMBER](#bucketing)                                   | Number of buckets to be created                              |
+| [BUCKETCOLUMNS](#bucketing)                                  | Columns which are to be placed in buckets                    |
 
  Following are the guidelines for TBLPROPERTIES, CarbonData's additional table options can be set via carbon.properties.
 
@@ -135,15 +151,15 @@ CarbonData DDL statements are documented here,which includes:
 
    ```
     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'
+      productNumber INT,
+      productName STRING,
+      storeCity STRING,
+      storeProvince STRING,
+      productCategory STRING,
+      productBatch STRING,
+      saleQuantity INT,
+      revenue INT)
+    STORED AS carbondata
     TBLPROPERTIES ('SORT_COLUMNS'='productName,storeCity',
                    'SORT_SCOPE'='NO_SORT')
    ```
@@ -222,10 +238,10 @@ CarbonData DDL statements are documented here,which includes:
 
 | Properties | Default value | Description |
 | ---------- | ------------- | ----------- |
-| LOCAL_DICTIONARY_ENABLE | false | Whether to enable local dictionary generation. **NOTE:** If this property is defined, it will override the value configured at system level by 'carbon.local.dictionary.enable' |
+| LOCAL_DICTIONARY_ENABLE | false | Whether to enable local dictionary generation. **NOTE:** If this property is defined, it will override the value configured at system level by '***carbon.local.dictionary.enable***'.Local dictionary will be generated for all string/varchar/char columns unless LOCAL_DICTIONARY_INCLUDE, LOCAL_DICTIONARY_EXCLUDE is configured. |
 | LOCAL_DICTIONARY_THRESHOLD | 10000 | The maximum cardinality of a column upto which carbondata can try to generate local dictionary (maximum - 100000) |
-| LOCAL_DICTIONARY_INCLUDE | string/varchar/char columns| Columns for which Local Dictionary has to be generated.**NOTE:** Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.|
-| LOCAL_DICTIONARY_EXCLUDE | none | Columns for which Local Dictionary need not be generated. |
+| LOCAL_DICTIONARY_INCLUDE | string/varchar/char columns| Columns for which Local Dictionary has to be generated.**NOTE:** Those string/varchar/char columns which are added into DICTIONARY_INCLUDE option will not be considered for local dictionary generation.This property needs to be configured only when local dictionary needs to be generated for few columns, skipping others.This property takes effect only when **LOCAL_DICTIONARY_ENABLE** is true or **carbon.local.dictionary.enable** is true |
+| LOCAL_DICTIONARY_EXCLUDE | none | Columns for which Local Dictionary need not be generated.This property needs to be configured only when local dictionary needs to be skipped for few columns, generating for others.This property takes effect only when **LOCAL_DICTIONARY_ENABLE** is true or **carbon.local.dictionary.enable** is true |
 
    **Fallback behavior:** 
 
@@ -252,7 +268,7 @@ CarbonData DDL statements are documented here,which includes:
              
                column3 LONG )
              
-     STORED BY 'carbondata'
+     STORED AS carbondata
      TBLPROPERTIES('LOCAL_DICTIONARY_ENABLE'='true','LOCAL_DICTIONARY_THRESHOLD'='1000',
      'LOCAL_DICTIONARY_INCLUDE'='column1','LOCAL_DICTIONARY_EXCLUDE'='column2')
    ```
@@ -407,7 +423,7 @@ CarbonData DDL statements are documented here,which includes:
 
   ```
   CREATE TABLE [IF NOT EXISTS] [db_name.]table_name 
-  STORED BY 'carbondata' 
+  STORED AS carbondata 
   [TBLPROPERTIES (key1=val1, key2=val2, ...)] 
   AS select_statement;
   ```
@@ -424,7 +440,7 @@ CarbonData DDL statements are documented here,which includes:
   carbon.sql("INSERT INTO source_table SELECT 2,'david','shenzhen',31")
   
   carbon.sql("CREATE TABLE target_table
-              STORED BY 'carbondata'
+              STORED AS carbondata
               AS SELECT city,avg(age) FROM source_table GROUP BY city")
               
   carbon.sql("SELECT * FROM target_table").show
@@ -441,7 +457,7 @@ CarbonData DDL statements are documented here,which includes:
   This function allows user to create external table by specifying location.
   ```
   CREATE EXTERNAL TABLE [IF NOT EXISTS] [db_name.]table_name 
-  STORED BY 'carbondata' LOCATION ‘$FilesPath’
+  STORED AS carbondata LOCATION ‘$FilesPath’
   ```
 
 ### Create external table on managed table data location.
@@ -450,14 +466,14 @@ CarbonData DDL statements are documented here,which includes:
 
   **Example:**
   ```
-  sql("CREATE TABLE origin(key INT, value STRING) STORED BY 'carbondata'")
+  sql("CREATE TABLE origin(key INT, value STRING) STORED AS carbondata")
   sql("INSERT INTO origin select 100,'spark'")
   sql("INSERT INTO origin select 200,'hive'")
   // creates a table in $storeLocation/origin
   
   sql(s"""
   |CREATE EXTERNAL TABLE source
-  |STORED BY 'carbondata'
+  |STORED AS carbondata
   |LOCATION '$storeLocation/origin'
   """.stripMargin)
   checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin"))
@@ -470,7 +486,7 @@ CarbonData DDL statements are documented here,which includes:
   **Example:**
   ```
   sql(
-  s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbondata' LOCATION
+  s"""CREATE EXTERNAL TABLE sdkOutputTable STORED AS carbondata LOCATION
   |'$writerPath' """.stripMargin)
   ```
 
@@ -670,7 +686,7 @@ Users can specify which columns to include and exclude for local dictionary gene
   ```
   CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type [COMMENT col_comment], ...)]
     [COMMENT table_comment]
-  STORED BY 'carbondata'
+  STORED AS carbondata
   [TBLPROPERTIES (property_name=property_value, ...)]
   ```
 
@@ -679,7 +695,7 @@ Users can specify which columns to include and exclude for local dictionary gene
   CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
                                 productNumber Int COMMENT 'unique serial number for product')
   COMMENT “This is table comment”
-   STORED BY 'carbondata'
+   STORED AS carbondata
    TBLPROPERTIES ('DICTIONARY_INCLUDE'='productNumber')
   ```
   You can also SET and UNSET table comment using ALTER command.
@@ -725,7 +741,7 @@ Users can specify which columns to include and exclude for local dictionary gene
                                 saleQuantity INT,
                                 revenue INT)
   PARTITIONED BY (productCategory STRING, productBatch STRING)
-  STORED BY 'carbondata'
+  STORED AS carbondata
   ```
    NOTE: Hive partition is not supported on complex datatype columns.
 		
@@ -780,7 +796,7 @@ Users can specify which columns to include and exclude for local dictionary gene
   CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
                     [(col_name data_type , ...)]
   PARTITIONED BY (partition_col_name data_type)
-  STORED BY 'carbondata'
+  STORED AS carbondata
   [TBLPROPERTIES ('PARTITION_TYPE'='HASH',
                   'NUM_PARTITIONS'='N' ...)]
   ```
@@ -796,7 +812,7 @@ Users can specify which columns to include and exclude for local dictionary gene
       col_D DECIMAL(10,2),
       col_F TIMESTAMP
   ) PARTITIONED BY (col_E LONG)
-  STORED BY 'carbondata' TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='9')
+  STORED AS carbondata TBLPROPERTIES('PARTITION_TYPE'='HASH','NUM_PARTITIONS'='9')
   ```
 
 ### Create Range Partition Table
@@ -806,7 +822,7 @@ Users can specify which columns to include and exclude for local dictionary gene
   CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
                     [(col_name data_type , ...)]
   PARTITIONED BY (partition_col_name data_type)
-  STORED BY 'carbondata'
+  STORED AS carbondata
   [TBLPROPERTIES ('PARTITION_TYPE'='RANGE',
                   'RANGE_INFO'='2014-01-01, 2015-01-01, 2016-01-01, ...')]
   ```
@@ -836,7 +852,7 @@ Users can specify which columns to include and exclude for local dictionary gene
   CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
                     [(col_name data_type , ...)]
   PARTITIONED BY (partition_col_name data_type)
-  STORED BY 'carbondata'
+  STORED AS carbondata
   [TBLPROPERTIES ('PARTITION_TYPE'='LIST',
                   'LIST_INFO'='A, B, C, ...')]
   ```
@@ -851,7 +867,7 @@ Users can specify which columns to include and exclude for local dictionary gene
       col_E LONG,
       col_F TIMESTAMP
    ) PARTITIONED BY (col_A STRING)
-   STORED BY 'carbondata'
+   STORED AS carbondata
    TBLPROPERTIES('PARTITION_TYPE'='LIST',
    'LIST_INFO'='aaaa, bbbb, (cccc, dddd), eeee')
   ```
@@ -914,7 +930,7 @@ Users can specify which columns to include and exclude for local dictionary gene
   ```
   CREATE TABLE [IF NOT EXISTS] [db_name.]table_name
                     [(col_name data_type, ...)]
-  STORED BY 'carbondata'
+  STORED AS carbondata
   TBLPROPERTIES('BUCKETNUMBER'='noOfBuckets',
   'BUCKETCOLUMNS'='columnname')
   ```
@@ -926,27 +942,16 @@ Users can specify which columns to include and exclude for local dictionary gene
   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'
+    productNumber INT,
+    saleQuantity INT,
+    productName STRING,
+    storeCity STRING,
+    storeProvince STRING,
+    productCategory STRING,
+    productBatch STRING,
+    revenue INT)
+  STORED AS carbondata
   TBLPROPERTIES ('BUCKETNUMBER'='4', 'BUCKETCOLUMNS'='productName')
   ```
 
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__docs').addClass('selected');
-
-  // Display docs subnav items
-  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
-    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>
 

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/dml-of-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/dml-of-carbondata.md b/src/site/markdown/dml-of-carbondata.md
index de23f5b..42da655 100644
--- a/src/site/markdown/dml-of-carbondata.md
+++ b/src/site/markdown/dml-of-carbondata.md
@@ -40,7 +40,31 @@ CarbonData DML statements are documented here,which includes:
   OPTIONS(property_name=property_value, ...)
   ```
 
-  **Supported Properties:** [DELIMITER](#delimiter), [QUOTECHAR](#quotechar), [COMMENTCHAR](#commentchar), [HEADER](#header), [FILEHEADER](#fileheader), [MULTILINE](#multiline), [ESCAPECHAR](#escapechar), [SKIP_EMPTY_LINE](#skip_empty_line), [COMPLEX_DELIMITER_LEVEL_1](#complex_delimiter_level_1), [COMPLEX_DELIMITER_LEVEL_2](#complex_delimiter_level_2), [ALL_DICTIONARY_PATH](#all_dictionary_path), [COLUMNDICT](#columndict), [DATEFORMAT](#dateformat),[ TIMESTAMPFORMAT](#timestampformat), [SORT_COLUMN_BOUNDS](#sort-column-bounds), [SINGLE_PASS](#single_pass), [BAD_RECORDS_LOGGER_ENABLE](#bad-records-handling), [BAD_RECORD_PATH](#bad-records-handling), [BAD_RECORDS_ACTION](#bad-records-handling), [IS_EMPTY_DATA_BAD_RECORD](#bad-records-handling), [GLOBAL_SORT_PARTITIONS](#global_sort_partitions)
+  **Supported Properties:**
+
+| Property                                                | Description                                                  |
+| ------------------------------------------------------- | ------------------------------------------------------------ |
+| [DELIMITER](#delimiter)                                 | Character used to separate the data in the input csv file    |
+| [QUOTECHAR](#quotechar)                                 | Character used to quote the data in the input csv file       |
+| [COMMENTCHAR](#commentchar)                             | Character used to comment the rows in the input csv file.Those rows will be skipped from processing |
+| [HEADER](#header)                                       | Whether the input csv files have header row                  |
+| [FILEHEADER](#fileheader)                               | If header is not present in the input csv, what is the column names to be used for data read from input csv |
+| [MULTILINE](#multiline)                                 | Whether a row data can span across multiple lines.           |
+| [ESCAPECHAR](#escapechar)                               | Escape character used to excape the data in input csv file.For eg.,\ is a standard escape character |
+| [SKIP_EMPTY_LINE](#skip_empty_line)                     | Whether empty lines in input csv file should be skipped or loaded as null row |
+| [COMPLEX_DELIMITER_LEVEL_1](#complex_delimiter_level_1) | Starting delimiter for complex type data in input csv file   |
+| [COMPLEX_DELIMITER_LEVEL_2](#complex_delimiter_level_2) | Ending delimiter for complex type data in input csv file     |
+| [ALL_DICTIONARY_PATH](#all_dictionary_path)             | Path to read the dictionary data from all columns            |
+| [COLUMNDICT](#columndict)                               | Path to read the dictionary data from for particular column  |
+| [DATEFORMAT](#dateformat)                               | Format of date in the input csv file                         |
+| [TIMESTAMPFORMAT](#timestampformat)                     | Format of timestamp in the input csv file                    |
+| [SORT_COLUMN_BOUNDS](#sort-column-bounds)               | How to parititon the sort columns to make the evenly distributed |
+| [SINGLE_PASS](#single_pass)                             | When to enable single pass data loading                      |
+| [BAD_RECORDS_LOGGER_ENABLE](#bad-records-handling)      | Whether to enable bad records logging                        |
+| [BAD_RECORD_PATH](#bad-records-handling)                | Bad records logging path.Useful when bad record logging is enabled |
+| [BAD_RECORDS_ACTION](#bad-records-handling)             | Behavior of data loading when bad record is found            |
+| [IS_EMPTY_DATA_BAD_RECORD](#bad-records-handling)       | Whether empty data of a column to be considered as bad record or not |
+| [GLOBAL_SORT_PARTITIONS](#global_sort_partitions)       | Number of partition to use for shuffling of data during sorting |
 
   You can use the following options to load data:
 
@@ -443,14 +467,3 @@ CarbonData DML statements are documented here,which includes:
   CLEAN FILES FOR TABLE carbon_table
   ```
 
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__docs').addClass('selected');
-
-  // Display docs subnav items
-  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
-    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/documentation.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/documentation.md b/src/site/markdown/documentation.md
index 66493bd..537a9d3 100644
--- a/src/site/markdown/documentation.md
+++ b/src/site/markdown/documentation.md
@@ -25,7 +25,7 @@ Apache CarbonData is a new big data file format for faster interactive query usi
 
 ## Getting Started
 
-**File Format Concepts:** Start with the basics of understanding the [CarbonData file format](./file-structure-of-carbondata.md#carbondata-file-structure) and its storage structure.This will help to understand other parts of the documentation, incuding deployment, programming and usage guides. 
+**File Format Concepts:** Start with the basics of understanding the [CarbonData file format](./file-structure-of-carbondata.md#carbondata-file-format) and its [storage structure](./file-structure-of-carbondata.md).This will help to understand other parts of the documentation, including deployment, programming and usage guides. 
 
 **Quick Start:** [Run an example program](./quick-start-guide.md#installing-and-configuring-carbondata-to-run-locally-with-spark-shell) on your local machine or [study some examples](https://github.com/apache/carbondata/tree/master/examples/spark2/src/main/scala/org/apache/carbondata/examples).
 
@@ -35,9 +35,9 @@ Apache CarbonData is a new big data file format for faster interactive query usi
 
 
 
-## Deployment
+## Integration
 
-CarbonData can be integrated with popular Execution engines like [Spark](./quick-start-guide.md#spark) and [Presto](./quick-start-guide.md#presto).Refer to the [Installation and Configuration](./quick-start-guide.md##deployment-modes) section to understand all modes of Integrating CarbonData.
+CarbonData can be integrated with popular Execution engines like [Spark](./quick-start-guide.md#spark) and [Presto](./quick-start-guide.md#presto).Refer to the [Installation and Configuration](./quick-start-guide.md#integration) section to understand all modes of Integrating CarbonData.
 
 
 
@@ -64,7 +64,3 @@ faster data format.Contributing to CarbonData doesn’t just mean writing code.
 
 **Trainings:** Training records on design and code flows can be found [here](https://cwiki.apache.org/confluence/display/CARBONDATA/CarbonData+Training+Materials).
 
-<script>
-// Show selected style on nav item
-$(function() { $('.b-nav__intro').addClass('selected'); });
-</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/faq.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/faq.md b/src/site/markdown/faq.md
index fdc2ca6..8ec7290 100644
--- a/src/site/markdown/faq.md
+++ b/src/site/markdown/faq.md
@@ -26,18 +26,18 @@
 * [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)
-* [Why all executors are showing success in Spark UI even after Dataload command failed at Driver side?](#Why-all-executors-are-showing-success-in-Spark-UI-even-after-Dataload-command-failed-at-driver-side)
-* [Why different time zone result for select query output when query SDK writer output?](#Why-different-time-zone-result-for-select-query-output-when-query-SDK-writer-output)
+* [Why all executors are showing success in Spark UI even after Dataload command failed at Driver side?](#why-all-executors-are-showing-success-in-spark-ui-even-after-dataload-command-failed-at-driver-side)
+* [Why different time zone result for select query output when query SDK writer output?](#why-different-time-zone-result-for-select-query-output-when-query-sdk-writer-output)
 
 # TroubleShooting
 
-- [Getting tablestatus.lock issues When loading data](#Getting-tablestatus.lock-issues-When-loading-data)
+- [Getting tablestatus.lock issues When loading data](#getting-tablestatuslock-issues-when-loading-data)
 - [Failed to load thrift libraries](#failed-to-load-thrift-libraries)
 - [Failed to launch the Spark Shell](#failed-to-launch-the-spark-shell)
 - [Failed to execute load query on cluster](#failed-to-execute-load-query-on-cluster)
 - [Failed to execute insert query on cluster](#failed-to-execute-insert-query-on-cluster)
 - [Failed to connect to hiveuser with thrift](#failed-to-connect-to-hiveuser-with-thrift)
-- [Failed to read the metastore db during table](#failed-to-read-the-metastore-db-during-table)
+- [Failed to read the metastore db during table creation](#failed-to-read-the-metastore-db-during-table-creation)
 - [Failed to load data on the cluster](#failed-to-load-data-on-the-cluster)
 - [Failed to insert data on the cluster](#failed-to-insert-data-on-the-cluster)
 - [Failed to execute Concurrent Operations(Load,Insert,Update) on table by multiple workers](#failed-to-execute-concurrent-operations-on-table-by-multiple-workers)
@@ -98,7 +98,7 @@ The property carbon.lock.type configuration specifies the type of lock to be acq
 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).
+Carbon support insert operation, you can refer to the syntax mentioned in [DML Operations on CarbonData](./dml-of-carbondata.md).
 First, create a source table in spark-sql and load data into this created table.
 
 ```
@@ -126,7 +126,7 @@ CREATE TABLE IF NOT EXISTS carbon_table(
 id String,
 city String,
 name String)
-STORED BY 'carbondata';
+STORED AS carbondata;
 ```
 
 ```
@@ -170,7 +170,7 @@ When SubQuery predicate is present in the query.
 Example:
 
 ```
-create table gdp21(cntry smallint, gdp double, y_year date) stored by 'carbondata';
+create table gdp21(cntry smallint, gdp double, y_year date) stored as carbondata;
 create datamap ag1 on table gdp21 using 'preaggregate' as select cntry, sum(gdp) from gdp21 group by cntry;
 select ctry from pop1 where ctry in (select cntry from gdp21 group by cntry);
 ```
@@ -181,7 +181,7 @@ When aggregate function along with 'in' filter.
 Example:
 
 ```
-create table gdp21(cntry smallint, gdp double, y_year date) stored by 'carbondata';
+create table gdp21(cntry smallint, gdp double, y_year date) stored as carbondata;
 create datamap ag1 on table gdp21 using 'preaggregate' as select cntry, sum(gdp) from gdp21 group by cntry;
 select cntry, sum(gdp) from gdp21 where cntry in (select ctry from pop1) group by cntry;
 ```
@@ -192,7 +192,7 @@ When aggregate function having 'join' with equal filter.
 Example:
 
 ```
-create table gdp21(cntry smallint, gdp double, y_year date) stored by 'carbondata';
+create table gdp21(cntry smallint, gdp double, y_year date) stored as carbondata;
 create datamap ag1 on table gdp21 using 'preaggregate' as select cntry, sum(gdp) from gdp21 group by cntry;
 select cntry,sum(gdp) from gdp21,pop1 where cntry=ctry group by cntry;
 ```
@@ -462,10 +462,3 @@ Note :  Refrain from using "mvn clean package" without specifying the profile.
   A single column that can be considered as dimension is mandatory for table creation.
 
 
-
-
-<script>
-// Show selected style on nav item
-$(function() { $('.b-nav__faq').addClass('selected'); });
-</script>
-

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/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
index 80c91d5..2b43105 100644
--- a/src/site/markdown/file-structure-of-carbondata.md
+++ b/src/site/markdown/file-structure-of-carbondata.md
@@ -6,48 +6,173 @@
     (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
+```
+  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.
+```
 
-    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 table 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.
+This document describes the what a CarbonData table looks like in a HDFS directory, files written and content of each file.
+
+- [File Directory Structure](#file-directory-structure)
+
+- [File Content details](#file-content-details)
+  - [Schema file format](#schema-file-format)
+  - [CarbonData file format](#carbondata-file-format)
+    - [Blocklet format](#blocklet-format)
+      - [V1](#v1)
+      - [V2](#v2)
+      - [V3](#v3)
+    - [Footer format](#footer-format)
+  - [carbonindex file format](#carbonindex-file-format)
+  - [Dictionary file format](#dictionary-file-format)
+  - [tablestatus file format](#tablestatus-file-format)
+
+## File Directory Structure
+
+The CarbonData files are stored in the location specified by the ***carbon.storelocation*** configuration (configured in carbon.properties; if not configured, the default is ../carbon.store).
+
+  The file directory structure is as below: 
+
+![File Directory Structure](../../src/site/images/2-1_1.png)
+
+1. ModifiedTime.mdt records the timestamp of the metadata with the modification time attribute of the file. When the drop table and create table are used, the modification time of the file is updated.This is common to all databases and hence is kept in parallel to databases
+2. The **default** is the database name and contains the user tables.default is used when user doesn't specify any database name;else user configured database name will be the directory name. user_table is the table name.
+3. Metadata directory stores schema files, tablestatus and dictionary files (including .dict, .dictmeta and .sortindex). There are three types of metadata data information files.
+4. data and index files are stored under directory named **Fact**. The Fact directory has a Part0 partition directory, where 0 is the partition number.
+5. There is a Segment_0 directory under the Part0 directory, where 0 is the segment number.
+6. There are two types of files, carbondata and carbonindex, in the Segment_0 directory.
+
+
+
+## File Content details
+
+When the table is created, the user_table directory is generated, and a schema file is generated in the Metadata directory for recording the table structure.
+
+When loading data in batches, each batch loading generates a new segment directory. The scheduling tries to control a task processing data loading task on each node. Each task will generate multiple carbondata files and one carbonindex file.
+
+During  global dictionary generation, if the two-pass scheme is used, before the data is loaded, the corresponding dict, dictmeta and sortindex files are generated for each dictionary-encoded column, and partial dictionary files can be provided by the pre-define dictionary method to reduce the need. A dictionary-encoded column is generated by scanning the full amount of data; a dictionary file of all dictionary code columns can also be provided by the all dictionary method to avoid scanning data. If the single-pass scheme is adopted, the global dictionary code is generated in real time during data loading, and after the data is loaded, the dictionary is solidified into a dictionary file.
+
+The following sections use the Java object generated by the thrift file describing the carbondata file format to explain the contents of each file one by one (you can also directly read the format defined in the [thrift file](https://github.com/apache/carbondata/tree/master/format/src/main/thrift))
+
+### Schema file format
+
+The contents of the schema file is as shown below
+
+![Schema file format](../../src/site/images/2-2_1.png)
+
+1. TableSchema class
+    The TableSchema class does not store the table name, it is infered from the directory name(user_table).
+    tableProperties is used to record table-related properties, such as: table_blocksize.
+2. ColumnSchema class
+    Encoders are used to record the encoding used in column storage.
+    columnProperties is used to record column related properties.
+3. BucketingInfo class
+    When creating a bucket table, you can specify the number of buckets in the table and the column to splitbuckets.
+4. DataType class
+    Describes the data types supported by CarbonData.
+5. Encoding class
+    Several encodings that may be used in CarbonData files.
+
+### CarbonData file format
+
+#### File Header
+
+It contains CarbonData file version number, list of column schema and schema updation timestamp.
+
+![File Header](../../src/site/images/carbon_data_file_structure_new.png)
+
+The carbondata file consists of multiple blocklets and footer parts. The blocklet is the dataset inside the carbondata file (the latest V3 format, the default configuration is 64MB), each blocklet contains a ColumnChunk for each column, and a ColumnChunk may contain one or more Column Pages.
+
+The carbondata file currently supports V1, V2 and V3 versions. The main difference is the change of the blocklet part, which is introduced one by one.
+
+#### Blocklet format
+
+#####  V1
+
+ Blocket consists of all column data pages, RLE pages, and rowID pages. Since the pages in the blocklet are grouped according to the page type, the three pieces of data of each column are distributed and stored in the blocklet, and the offset and length information of all the pages need to be recorded in the footer part.
+
+![V1](../../src/site/images/2-3_1.png)
+
+##### V2
+
+The blocklet consists of ColumnChunk for all columns. The ColumnChunk for a column consists of a ColumnPage, which includes the data chunk header, data page, RLE page, and rowID page. Since ColumnChunk aggregates the three types of Page data of the column together, it can read the column data using fewer readers. Since the header part records the length information of all the pages, the footer part only needs to record the offset and length of the ColumnChunk, and also reduces the amount of footer data.
+
+![V2](../../src/site/images/2-3_2.png)
+
+##### V3
+
+The blocklet is also composed of ColumnChunks of all columns. What is changed is that a ColumnChunk consists of one or more Column Pages, and Column Page adds a new BlockletMinMaxIndex.
+
+Compared with V2: The blocklet data volume of V2 format defaults to 120,000 lines, and the blocklet data volume of V3 format defaults to 64MB. For the same size data file, the information of the footer part index metadata may be further reduced; meanwhile, the V3 format adds a new page. Level data filtering, and the amount of data per page is only 32,000 lines by default, which is much less than the 120,000 lines of V2 format. The accuracy of data filtering hits further, and more data can be filtered out before decompressing data.
+
+![V3](../../src/site/images/2-3_3.png)
+
+#### Footer format
+
+Footer records each carbondata
+All blocklet data distribution information and statistical related metadata information (minmax, startkey/endkey) inside the file.
+
+![Footer format](../../src/site/images/2-3_4.png)
+
+1.  BlockletInfo3 is used to record the offset and length of all ColumnChunk3.
+2.  SegmentInfo is used to record the number of columns and the cardinality of each column.
+3.  BlockletIndex includes BlockletMinMaxIndex and BlockletBTreeIndex.
+
+BlockletBTreeIndex is used to record the startkey/endkey of all blocklets in the block. When querying, the startkey/endkey of the query is generated by filtering conditions combined with mdkey. With BlocketBtreeIndex, the range of blocklets satisfying the conditions in each block can be delineated.
+
+BlockletMinMaxIndex is used to record the min/max value of all columns in the blocklet. By using the min/max check on the filter condition, you can skip the block/blocklet that does not satisfy the condition.
+
+### carbonindex file format
+
+Extract the BlockletIndex part of the footer part to generate the carbonindex file. Load data in batches, schedule as much as possible to control a node to start a task, each task generates multiple carbondata files and a carbonindex file. The carbonindex file records the index information of all the blocklets in all the carbondata files generated by the task.
+
+As shown in the figure, the index information corresponding to a block is recorded by a BlockIndex object, including carbondata filename, footer offset and BlockletIndex. The BlockIndex data volume is less than the footer. The file is directly used to build the index on the driver side when querying, without having to skip the footer part of the data volume of multiple data files.
+
+![carbonindex file format](../../src/site/images/2-4_1.png)
+
+### Dictionary file format
+
+
+For each dictionary encoded column, a dictionary file is used to store the dictionary metadata for that column.
+
+1. dict file records the distinct value list of a column
+
+For the first time dataloading, the file is generated using a distinct value list of a column. The value in the file is unordered; the subsequent append is used. In the second step of dataloading (Data Convert Step), the dictionary code column will replace the true value of the data with the dictionary key.
+
+![Dictionary file format](../../src/site/images/2-5_1.png)
+
+
+2.  dictmeta records the metadata description of the new distinct value of each dataloading
+
+The dictionary cache uses this information to incrementally flush the cache.
+
+![Dictionary Chunk](../../src/site/images/2-5_2.png)
+	
+
+3.  sortindex records the result set of the key code of the dictionary code sorted by value.
 
-![CarbonData File Structure](../../src/site/images/carbon_data_file_structure_new.png)
+In dataLoading, if there is a new dictionary value, the sortindex file will be regenerated using all the dictionary codes.
 
-* 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.
+Filtering queries based on dictionary code columns need to convert the value filter filter to the key filter condition. Using the sortindex file, you can quickly construct an ordered value sequence to quickly find the key value corresponding to the value, thus speeding up the conversion process.
 
-![CarbonData File Format](../../src/site/images/carbon_data_format_new.png)
+![sortindex file format](../../src/site/images/2-5_3.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.
+### tablestatus file format
 
+Tablestatus records the segment-related information (in gson format) for each load and merge, including load time, load status, segment name, whether it was deleted, and the segment name incorporated. Regenerate the tablestatusfile after each load or merge.
 
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__docs').addClass('selected');
+![tablestatus file format](../../src/site/images/2-6_1.png)
 
-  // Display docs subnav items
-  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
-    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/how-to-contribute-to-apache-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/how-to-contribute-to-apache-carbondata.md b/src/site/markdown/how-to-contribute-to-apache-carbondata.md
index 741e6d6..f64c948 100644
--- a/src/site/markdown/how-to-contribute-to-apache-carbondata.md
+++ b/src/site/markdown/how-to-contribute-to-apache-carbondata.md
@@ -189,11 +189,4 @@ From another local branch, run:
 $ git fetch --all
 $ git branch -d <my-branch>
 $ git push <GitHub_user> --delete <my-branch>
-```
-
-
-<script>
-// Show selected style on nav item
-$(function() { $('.b-nav__contri').addClass('selected'); });
-</script>
-
+```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/introduction.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/introduction.md b/src/site/markdown/introduction.md
index 8169958..434ccfa 100644
--- a/src/site/markdown/introduction.md
+++ b/src/site/markdown/introduction.md
@@ -16,157 +16,102 @@ CarbonData has
 
 - **Multi level indexing** to efficiently prune the files and data to be scanned and hence reduce I/O scans and CPU processing
 
+## CarbonData Features & Functions
 
+CarbonData has rich set of featues to support various use cases in Big Data analytics.The below table lists the major features supported by CarbonData.
 
-## Architecture
 
-![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_architecture.png)
 
+### Table Management
 
+- ##### DDL (Create, Alter,Drop,CTAS)
 
-#### Spark Interface Layer: 
+​	CarbonData provides its own DDL to create and manage carbondata tables.These DDL conform to 			Hive,Spark SQL format and support additional properties and configuration to take advantages of CarbonData functionalities.
 
-CarbonData has deep integration with Apache Spark.CarbonData integrates custom Parser,Strategies,Optimization rules into Spark to take advantage of computing performed closer to data.
+- ##### DML(Load,Insert)
 
-![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_spark_integration.png)
+  CarbonData provides its own DML to manage data in carbondata tables.It adds many customizations through configurations to completely customize the behavior as per user requirement scenarios.
 
-1. **Carbon parser** Enhances Spark’s SQL parser to support Carbon specific DDL and DML commands to create carbon table, create aggregate tables, manage data loading, data retention and cleanup.
-2. **Carbon Strategies**:- Modify Spark SQL’s physical query execution plan to push down possible operations to Carbon for example:- Grouping, Distinct Count, Top N etc.. for improving query performance.
-3. **Carbon Data RDD**:- Makes the data present in Carbon tables visible to Spark as a RDD which enables spark to perform distributed computation on Carbon tables.
+- ##### Update and Delete
 
+  CarbonData supports Update and Delete on Big Data.CarbonData provides the syntax similar to Hive to support IUD operations on CarbonData tables.
 
+- ##### Segment Management
 
-#### Carbon Processor: 
+  CarbonData has unique concept of segments to manage incremental loads to CarbonData tables effectively.Segment management helps to easily control the table, perform easy retention, and is also used to provide transaction capability for operations being performed.
 
-Receives a query execution fragment from spark and executes the same on the Carbon storage. This involves Scanning the carbon store files for matching record, using the indices to directly locate the row sets and even the rows that may containing the data being searched for. The Carbon processor also performs all pushed down operations such as 
+- ##### Partition
 
-Aggregation/Group By
+  CarbonData supports 2 kinds of partitions.1.partition similar to hive partition.2.CarbonData partition supporting hash,list,range partitioning.
 
-Distinct Count
+- ##### Compaction
 
-Top N
+  CarbonData manages incremental loads as segments.Compaction help to compact the growing number of segments and also to improve query filter pruning.
 
-Expression Evaluation
+- ##### External Tables
 
-And many more…
+  CarbonData can read any carbondata file and automatically infer schema from the file and provide a relational table view to perform sql queries using Spark or any other applicaion.
 
-#### Carbon Storage:
+### DataMaps
 
-Custom columnar data store which is heavily compressed, binary, dictionary encoded and heavily indexed.Usaually stored in HDFS.
+- ##### Pre-Aggregate
 
-## CarbonData Features
+  CarbonData has concept of datamaps to assist in pruning of data while querying so that performance is faster.Pre Aggregate tables are kind of datamaps which can improve the query performance by order of magnitude.CarbonData will automatically pre-aggregae the incremental data and re-write the query to automatically fetch from the most appropriate pre-aggregate table to serve the query faster.
 
-CarbonData has rich set of featues to support various use cases in Big Data analytics.
+- ##### Time Series
 
- 
+  CarbonData has built in understanding of time order(Year, month,day,hour, minute,second).Time series is a pre-aggregate table which can automatically roll-up the data to the desired level during incremental load and serve the query from the most appropriate pre-aggregate table.
 
-## Design
+- ##### Bloom filter
 
-- ### Dictionary Encoding
+  CarbonData supports bloom filter as a datamap in order to quickly and efficiently prune the data for scanning and acheive faster query performance.
 
-CarbonData supports encoding of data with suggogate values to reduce storage space and speed up processing.Most databases and big data SQL data stores adopt dictionary encoding(integer surrogate numbers) to achieve data compression.Unlike other column store databases where the dictionary is local to each data block, CarbonData maintains a global dictionary which provides opportunity for lazy conversion to actual values enabling all computation to be performed on the lightweight surrogate values.
+- ##### Lucene
 
-##### Dictionary generation
+  Lucene is popular for indexing text data which are long.CarbonData provides a lucene datamap so that text columns can be indexed using lucene and use the index result for efficient pruning of data to be retrieved during query.
 
-![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_dict_encoding.png)
+- ##### MV (Materialized Views)
 
+  MVs are kind of pre-aggregate tables which can support efficent query re-write and processing.CarbonData provides MV which can rewrite query to fetch from any table(including non-carbondata tables).Typical usecase is to store the aggregated data of a non-carbondata fact table into carbondata and use mv to rewrite the query to fetch from carbondata.
 
+### Streaming
 
-##### MDK Indexing
+- ##### Spark Streaming
 
-All the surrogate keys are byte packed to generate an MDK (Multi Dimensional Key) Index.
+  CarbonData supports streaming of data into carbondata in near-realtime and make it immediately available for query.CarbonData provides a DSL to create source and sink tables easily without the need for the user to write his application.
 
-Any non surrogate columns of String data types are compressed using one of the configured compression algorithms and stored.For those numeric columns where surrogates are not generated, such data is stored as it is after compression.
+### SDK
 
-![image-20180903212418381](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_mdk.png)
+- ##### CarbonData writer
 
-##### Sorted MDK
+  CarbonData supports writing data from non-spark application using SDK.Users can use SDK to generate carbondata files from custom applications.Typical usecase is to write the streaming application plugged in to kafka and use carbondata as sink(target) table for storing.
 
-The data is sorted based on the MDK Index.Sorting helps for logical grouping of similar data and there by aids in faster look up during query.
+- ##### CarbonData reader
 
-#### ![image-20180903212525214](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_mdk_sort.png)
+  CarbonData supports reading of data from non-spark application using SDK.Users can use the SDK to read the carbondata files from their application and do custom processing.
 
-##### Custom Columnar Encoding
+### Storage
 
-The Sorted MDK Index is split into each column.Unlike other stores where the column is compressed and stored as it is, CarbonData sorts this column data so that Binary Search can be performed on individual column data based on the filter conditions.This aids in magnitude increase in query performance and also in better compression.Since the individual column's data gets sorted, it is necessary to maintain the row mapping with the sorted MDK Index data in order to retrieve data from other columns which are not participating in filter.This row mapping is termed as **Inverted Index** and is stored along with the column data.The below picture depicts the logical column view.User has the option to **turn off** Inverted Index for such columns where filters are never applied or is very rare.In such cases, scanning would be sequential, but can aid in reducing the storage size(occupied due to inverted index data).
+- ##### S3
 
-#### ![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_blocklet_view.png)
+  CarbonData can write to S3, OBS or any cloud storage confirming to S3 protocol.CarbonData uses the HDFS api to write to cloud object stores.
 
-- ### CarbonData Storage Format
+- ##### HDFS
 
-  CarbonData has a unique storage structure which aids in efficient storage and retrieval of data.Please refer to [File Structure of CarbonData](#./file-structure-of-carbondata.md) for detailed information on the format.
-
-- ### Indexing
-
-  CarbonData maintains multiple indexes at multiple levels to assist in efficient pruning of unwanted data from scan during query.Also CarbonData has support for plugging in external indexing solutions to speed up the query process.
-
-  ##### Min-Max Indexing
-
-  Storing data along with index significantly accelerates query performance and reduces the I/O scans and CPU resources in case of filters in the query. CarbonData index consists of multiple levels of indices, a processing framework can leverage this index to reduce the number of tasks it needs to schedule and process. It can also do skip scan in more fine grained units (called blocklet) in task side scanning instead of scanning the whole file.  **CarbonData maintains Min-Max Index for all the columns.**
-
-  CarbonData maintains a separate index file which contains the footer information for efficient IO reads.
-
-  Using the Min-Max info in these index files, two levels of filtering can be achieved.
-
-  Min-Max at the carbondata file level,to efficiently prune the files when the filter condition doesn't fall in the range.This information when maintained at the Spark Driver, will help to efficiently schedule the tasks for scanning
-
-  Min-Max at the blocklet level, to efficiently prune the blocklets when the filter condition doesn't fall in the range.This information when maintained at the executor can significantly reduce the amount unnecessary data processed by the executor tasks. 
-
-
-
-  ![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-minmax-blocklet.png)
-
-- #### DataMaps
-
-  DataMap is a framework for indexing and also for statistics that can be used to add primary index (Blocklet Index) , secondary index type and statistical type to CarbonData.
-
-  DataMap is a standardized general interface which CarbonData uses to prune data blocks for scanning.
-
-  DataMaps are of 2 types:
-
-  **CG(Coarse Grained) DataMaps** Can prune data to the blocklet or to Page level.ie., Holds information for deciding which blocks/blocklets to be scanned.This DataMap is used in Spark Driver to decide the number of tasks to be scheduled.
-
-  **FG(Fine Grained) DataMaps** Can prune data to row level.This DataMap is used in Spark executor for scanning an fetching the data much faster.
-
-  Since DataMap interfaces are generalised, We can write a thin adaptor called as **DataMap Providers** to interface between CarbonData and other external Indexing engines. For eg., Lucene, Solr,ES,...
-
-  CarbonData has its own DSL to create and manage DataMaps.Please refer to [CarbonData DSL](#./datamap/datamap-management.md#overview) for more information.
-
-  The below diagram explains about the DataMap execution in CarbonData.
-
-  ![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-datamap.png)
-
-- #### Update & Delete
-
-
-CarbonData supports Update and delete operations over big data.This functionality is not targetted for OLTP scenarios where high concurrent update/delete are required.Following are the assumptions considered when this feature is designed.
-
-1. Updates or Deletes are periodic and in Bulk
-2. Updates or Deletes are atomic
-3. Data is immediately visible
-4. Concurrent query to be allowed during an update or delete operation
-5. Single statement auto-commit support (not OLTP-style transaction)
-
-Since data stored in HDFS are immutable,data blocks cannot be updated in-place.Re-write of entire data block is not efficient for IO and also is a slow process.
-
-To over come these limitations, CarbonData adopts methodology of writing a delta file containing the rows to be deleted and another delta file containing the values to be updated with.During processing, These two delta files are merged with the main carbondata file and the correct result is returned for the query.
-
-The below diagram describes the process.
-
-![](/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_update_delete.png)
+  CarbonData uses HDFS api to write and read data from HDFS.CarbonData can take advantage of the locality information to efficiently suggest spark to run tasks near to the data.
 
 
 
 ## Integration with Big Data ecosystem
 
-Refer to Integration with [Spark](#./quick-start-guide.md#spark), [Presto](#./quick-start-guide.md#presto) for detailed information on integrating CarbonData with these execution engines.
+Refer to Integration with [Spark](./quick-start-guide.md#spark), [Presto](./quick-start-guide.md#presto) for detailed information on integrating CarbonData with these execution engines.
 
 ## Scenarios where CarbonData is suitable
 
+CarbonData is useful in various analytical work loads.Some of the most typical usecases where CarbonData is being used is [documented here](./usecases.md).
+
 
 
+## Performance Results
 
-<script>
-// Show selected style on nav item
-$(function() { $('.b-nav__intro').addClass('selected'); });
-</script>
\ No newline at end of file
+![Performance Results](../docs/images/carbondata-performance.png?raw=true)

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/language-manual.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/language-manual.md b/src/site/markdown/language-manual.md
index 9fef71b..123cae3 100644
--- a/src/site/markdown/language-manual.md
+++ b/src/site/markdown/language-manual.md
@@ -24,12 +24,11 @@ CarbonData has its own parser, in addition to Spark's SQL Parser, to parse and p
 - [Data Types](./supported-data-types-in-carbondata.md)
 - Data Definition Statements
   - [DDL:](./ddl-of-carbondata.md)[Create](./ddl-of-carbondata.md#create-table),[Drop](./ddl-of-carbondata.md#drop-table),[Partition](./ddl-of-carbondata.md#partition),[Bucketing](./ddl-of-carbondata.md#bucketing),[Alter](./ddl-of-carbondata.md#alter-table),[CTAS](./ddl-of-carbondata.md#create-table-as-select),[External Table](./ddl-of-carbondata.md#create-external-table)
-  - Indexes
-  - [DataMaps](./datamap-management.md)
-    - [Bloom](./bloomfilter-datamap-guide.md)
-    - [Lucene](./lucene-datamap-guide.md)
-    - [Pre-Aggregate](./preaggregate-datamap-guide.md)
-    - [Time Series](./timeseries-datamap-guide.md)
+  - [DataMaps](./datamap/datamap-management.md)
+    - [Bloom](./datamap/bloomfilter-datamap-guide.md)
+    - [Lucene](./datamap/lucene-datamap-guide.md)
+    - [Pre-Aggregate](./datamap/preaggregate-datamap-guide.md)
+    - [Time Series](./datamap/timeseries-datamap-guide.md)
   - Materialized Views (MV)
   - [Streaming](./streaming-guide.md)
 - Data Manipulation Statements
@@ -37,15 +36,4 @@ CarbonData has its own parser, in addition to Spark's SQL Parser, to parse and p
   - [Segment Management](./segment-management-on-carbondata.md)
 - [Configuration Properties](./configuration-parameters.md)
 
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__docs').addClass('selected');
-
-  // Display docs subnav items
-  if (!$('.b-nav__docs').parent().hasClass('nav__item__with__subs--expanded')) {
-    $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>
 

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/lucene-datamap-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/lucene-datamap-guide.md b/src/site/markdown/lucene-datamap-guide.md
index 248c8e5..86b00e2 100644
--- a/src/site/markdown/lucene-datamap-guide.md
+++ b/src/site/markdown/lucene-datamap-guide.md
@@ -59,7 +59,7 @@ It will show all DataMaps created on main table.
     age int,
     city string,
     country string)
-  STORED BY 'carbondata'
+  STORED AS carbondata
   ```
   
   User can create Lucene datamap using the Create DataMap DDL:
@@ -149,7 +149,7 @@ select * from datamap_test where TEXT_MATCH('name:*n*')
 
 select * from datamap_test where TEXT_MATCH('name:*10 -name:*n*')
 ```
-**Note:** For lucene queries and syntax, refer to [lucene-syntax](www.lucenetutorial.com/lucene-query-syntax.html)
+**Note:** For lucene queries and syntax, refer to [lucene-syntax](http://www.lucenetutorial.com/lucene-query-syntax.html)
 
 ## Data Management with lucene datamap
 Once there is lucene datamap is created on the main table, following command on the main
@@ -173,15 +173,4 @@ release, user can do as following:
 3. Create the lucene datamap again by `CREATE DATAMAP` command.
 Basically, user can manually trigger the operation by re-building the datamap.
 
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__datamap').addClass('selected');
-  
-  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
-    // Display datamap subnav items
-    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>
 

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/performance-tuning.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/performance-tuning.md b/src/site/markdown/performance-tuning.md
index d8b53f2..f56a63b 100644
--- a/src/site/markdown/performance-tuning.md
+++ b/src/site/markdown/performance-tuning.md
@@ -22,6 +22,7 @@
   * [Suggestions to create CarbonData Table](#suggestions-to-create-carbondata-table)
   * [Configuration for Optimizing Data Loading performance for Massive Data](#configuration-for-optimizing-data-loading-performance-for-massive-data)
   * [Optimizing Query Performance](#configurations-for-optimizing-carbondata-performance)
+  * [Compaction Configurations for Optimizing CarbonData Query Performance](#compaction-configurations-for-optimizing-carbondata-query-performance)
 
 ## Suggestions to Create CarbonData Table
 
@@ -56,7 +57,7 @@
     counter_1, Decimal
     ...
     
-    )STORED BY 'carbondata'
+    )STORED AS carbondata
     TBLPROPERTIES ('SORT_COLUMNS'='msisdn, Dime_1')
   ```
 
@@ -81,7 +82,7 @@
       counter_1, Decimal
       ...
       
-      )STORED BY 'carbondata'
+      )STORED AS carbondata
       TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
   ```
 
@@ -100,7 +101,7 @@
     counter_1 decimal,
     counter_2 double,
     ...
-    )STORED BY 'carbondata'
+    )STORED AS carbondata
     TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
 ```
   The result of performance analysis of test-case shows reduction in query execution time from 15 to 3 seconds, thereby improving performance by nearly 5 times.
@@ -121,12 +122,12 @@
     END_TIME bigint,
     ...
     counter_100 double
-    )STORED BY 'carbondata'
+    )STORED AS carbondata
     TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
   ```
 
   **NOTE:**
-  + BloomFilter can be created to enhance performance for queries with precise equal/in conditions. You can find more information about it in BloomFilter datamap [document](https://github.com/apache/carbondata/blob/master/docs/datamap/bloomfilter-datamap-guide.md).
+  + BloomFilter can be created to enhance performance for queries with precise equal/in conditions. You can find more information about it in BloomFilter datamap [document](./datamap/bloomfilter-datamap-guide.md).
 
 
 ## Configuration for Optimizing Data Loading performance for Massive Data
@@ -176,8 +177,70 @@
 
   Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.
 
+## Compaction Configurations for Optimizing CarbonData Query Performance
+
+CarbonData provides many configurations to tune the compaction behavior so that query peformance is improved.
+
+
+
+Based on the number of cores available in the node, it is recommended to tune the configuration 	***carbon.number.of.cores.while.compacting*** appropriately.Configuring a higher value will improve the overall compaction performance.
+
+<p>&nbsp;</p>
+<table style="width: 777px;">
+<tbody>
+<tr style="height: 23px;">
+<td style="height: 23px; width: 95.375px;">No</td>
+<td style="height: 23px; width: 299.625px;">&nbsp;Data Loading frequency</td>
+<td style="height: 23px; width: 144px;">Data Size of each load</td>
+<td style="height: 23px; width: 204px;">Minor Compaction configuration</td>
+<td style="height: 23px; width: 197px;">&nbsp;Major compaction configuration</td>
+</tr>
+<tr style="height: 29.5px;">
+<td style="height: 29.5px; width: 95.375px;">1</td>
+<td style="height: 29.5px; width: 299.625px;">&nbsp;Batch(Once is several Hours)</td>
+<td style="height: 29.5px; width: 144px;">Big</td>
+<td style="height: 29.5px; width: 204px;">&nbsp;Not Suggested</td>
+<td style="height: 29.5px; width: 197px;">Configure Major Compaction size of 3-4 load size.Perform Major compaction once in a day</td>
+</tr>
+<tr style="height: 23px;">
+<td style="height: 23px; width: 95.375px;" rowspan="2">2</td>
+<td style="height: 23px; width: 299.625px;" rowspan="2">&nbsp;Batch(Once in few minutes)&nbsp;</td>
+<td style="height: 23px; width: 144px;">Big&nbsp;</td>
+<td style="height: 23px; width: 204px;">
+<p>&nbsp;Minor compaction (2,2).</p>
+<p>Enable Auto compaction, if high rate data loading speed is not required or the time between loads is sufficient to run the compaction</p>
+</td>
+<td style="height: 23px; width: 197px;">Major compaction size of 10 load size.Perform Major compaction once in a day</td>
+</tr>
+<tr style="height: 23px;">
+<td style="height: 23px; width: 144px;">Small</td>
+<td style="height: 23px; width: 204px;">
+<p>Minor compaction (6,6).</p>
+<p>Enable Auto compaction, if high rate data loading speed is not required or the time between loads is sufficient to run the compaction</p>
+</td>
+<td style="height: 23px; width: 197px;">Major compaction size of 10 load size.Perform Major compaction once in a day</td>
+</tr>
+<tr style="height: 23px;">
+<td style="height: 23px; width: 95.375px;">3</td>
+<td style="height: 23px; width: 299.625px;">&nbsp;History data loaded as single load,incremental loads matches&nbsp;(1) or (2)</td>
+<td style="height: 23px; width: 144px;">Big</td>
+<td style="height: 23px; width: 204px;">
+<p>&nbsp;Configure ALLOWED_COMPACTION_DAYS to exclude the History load.</p>
+<p>Configure Minor compaction configuration based&nbsp;condition (1) or (2)</p>
+</td>
+<td style="height: 23px; width: 197px;">&nbsp;Configure Major compaction size smaller than the history load size.</td>
+</tr>
+<tr style="height: 23px;">
+<td style="height: 23px; width: 95.375px;">4</td>
+<td style="height: 23px; width: 299.625px;">&nbsp;There can be error in recent data loaded.Need reload sometimes</td>
+<td style="height: 23px; width: 144px;">&nbsp;(1) or (2)</td>
+<td style="height: 23px; width: 204px;">
+<p>&nbsp;Configure COMPACTION_PRESERVE_SEGMENTS</p>
+<p>to exclude the recent few segments from compacting.</p>
+<p>Configure Minor compaction configuration based&nbsp;condition (1) or (2)</p>
+</td>
+<td style="height: 23px; width: 197px;">Same as (1) or (2)&nbsp;</td>
+</tr>
+</tbody>
+</table>
 
-<script>
-// Show selected style on nav item
-$(function() { $('.b-nav__perf').addClass('selected'); });
-</script>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/site/markdown/preaggregate-datamap-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/preaggregate-datamap-guide.md b/src/site/markdown/preaggregate-datamap-guide.md
index 9c7a5f8..3a3efc2 100644
--- a/src/site/markdown/preaggregate-datamap-guide.md
+++ b/src/site/markdown/preaggregate-datamap-guide.md
@@ -64,7 +64,7 @@ Start spark-shell in new terminal, type :paste, then copy and run the following
       | country string,
       | quantity int,
       | price bigint)
-      | STORED BY 'carbondata'
+      | STORED AS carbondata
     """.stripMargin)
  
  // Create pre-aggregate table on the main table
@@ -162,7 +162,7 @@ It will show all DataMaps created on main table.
     country string,
     quantity int,
     price bigint)
-  STORED BY 'carbondata'
+  STORED AS carbondata
   ```
   
   User can create pre-aggregate tables using the Create DataMap DDL
@@ -270,15 +270,3 @@ release, user can do as following:
 3. Create the pre-aggregate table again by `CREATE DATAMAP` command
 Basically, user can manually trigger the operation by re-building the datamap.
 
-
-<script>
-$(function() {
-  // Show selected style on nav item
-  $('.b-nav__datamap').addClass('selected');
-  
-  if (!$('.b-nav__datamap').parent().hasClass('nav__item__with__subs--expanded')) {
-    // Display datamap subnav items
-    $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
-  }
-});
-</script>


[24/39] carbondata-site git commit: Handled comments

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/installation-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/installation-guide.html b/src/main/webapp/installation-guide.html
deleted file mode 100644
index 696db00..0000000
--- a/src/main/webapp/installation-guide.html
+++ /dev/null
@@ -1,455 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
-                                   target="_blank">Apache CarbonData 1.4.1</a></li>
-							<li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
-                                   target="_blank">Apache CarbonData 1.4.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
-                                   target="_blank">Apache CarbonData 1.3.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
-                                   target="_blank">Apache CarbonData 1.3.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search" target="_blank">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input"  placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="row">
-                                <div class="col-sm-12  col-md-12">
-                                    <div>
-<h1>
-<a id="installation-guide" class="anchor" href="#installation-guide" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installation Guide</h1>
-<p>This tutorial guides you through the installation and configuration of CarbonData in the following two modes :</p>
-<ul>
-<li><a href="#installing-and-configuring-carbondata-on-standalone-spark-cluster">Installing and Configuring CarbonData on Standalone Spark Cluster</a></li>
-<li><a href="#installing-and-configuring-carbondata-on-spark-on-yarn-cluster">Installing and Configuring CarbonData on Spark on YARN Cluster</a></li>
-</ul>
-<p>followed by :</p>
-<ul>
-<li><a href="#query-execution-using-carbondata-thrift-server">Query Execution using CarbonData Thrift Server</a></li>
-</ul>
-<h2>
-<a id="installing-and-configuring-carbondata-on-standalone-spark-cluster" class="anchor" href="#installing-and-configuring-carbondata-on-standalone-spark-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Standalone Spark Cluster</h2>
-<h3>
-<a id="prerequisites" class="anchor" href="#prerequisites" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h3>
-<ul>
-<li>
-<p>Hadoop HDFS and Yarn should be installed and running.</p>
-</li>
-<li>
-<p>Spark should be installed and running on all the cluster nodes.</p>
-</li>
-<li>
-<p>CarbonData user should have permission to access HDFS.</p>
-</li>
-</ul>
-<h3>
-<a id="procedure" class="anchor" href="#procedure" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Procedure</h3>
-<ol>
-<li>
-<p><a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>Build the CarbonData</a> project and get the assembly jar from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code>.</p>
-</li>
-<li>
-<p>Copy <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code> to <code>$SPARK_HOME/carbonlib</code> folder.</p>
-<p><strong>NOTE</strong>: Create the carbonlib folder if it does not exist inside <code>$SPARK_HOME</code> path.</p>
-</li>
-<li>
-<p>Add the carbonlib folder path in the Spark classpath. (Edit <code>$SPARK_HOME/conf/spark-env.sh</code> file and modify the value of <code>SPARK_CLASSPATH</code> by appending <code>$SPARK_HOME/carbonlib/*</code> to the existing value)</p>
-</li>
-<li>
-<p>Copy the <code>./conf/carbon.properties.template</code> file from CarbonData repository to <code>$SPARK_HOME/conf/</code> folder and rename the file to <code>carbon.properties</code>.</p>
-</li>
-<li>
-<p>Repeat Step 2 to Step 5 in all the nodes of the cluster.</p>
-</li>
-<li>
-<p>In Spark node[master], configure the properties mentioned in the following table in <code>$SPARK_HOME/conf/spark-defaults.conf</code> file.</p>
-</li>
-</ol>
-<table>
-<thead>
-<tr>
-<th>Property</th>
-<th>Value</th>
-<th>Description</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>spark.driver.extraJavaOptions</td>
-<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
-<td>A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.</td>
-</tr>
-<tr>
-<td>spark.executor.extraJavaOptions</td>
-<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
-<td>A string of extra JVM options to pass to executors. For instance, GC settings or other logging. <strong>NOTE</strong>: You can enter multiple values separated by space.</td>
-</tr>
-</tbody>
-</table>
-<ol start="7">
-<li>Add the following properties in <code>$SPARK_HOME/conf/carbon.properties</code> file:</li>
-</ol>
-<table>
-<thead>
-<tr>
-<th>Property</th>
-<th>Required</th>
-<th>Description</th>
-<th>Example</th>
-<th>Remark</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>carbon.storelocation</td>
-<td>NO</td>
-<td>Location where data CarbonData will create the store and write the data in its own format. If not specified then it takes spark.sql.warehouse.dir path.</td>
-<td>hdfs://HOSTNAME:PORT/Opt/CarbonStore</td>
-<td>Propose to set HDFS directory</td>
-</tr>
-</tbody>
-</table>
-<ol start="8">
-<li>Verify the installation. For example:</li>
-</ol>
-<pre><code>./spark-shell --master spark://HOSTNAME:PORT --total-executor-cores 2
---executor-memory 2G
-</code></pre>
-<p><strong>NOTE</strong>: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.</p>
-<p>To get started with CarbonData : <a href="quick-start-guide.html">Quick Start</a>, <a href="data-management-on-carbondata.html">Data Management on CarbonData</a></p>
-<h2>
-<a id="installing-and-configuring-carbondata-on-spark-on-yarn-cluster" class="anchor" href="#installing-and-configuring-carbondata-on-spark-on-yarn-cluster" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Installing and Configuring CarbonData on Spark on YARN Cluster</h2>
-<p>This section provides the procedure to install CarbonData on "Spark on YARN" cluster.</p>
-<h3>
-<a id="prerequisites-1" class="anchor" href="#prerequisites-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Prerequisites</h3>
-<ul>
-<li>Hadoop HDFS and Yarn should be installed and running.</li>
-<li>Spark should be installed and running in all the clients.</li>
-<li>CarbonData user should have permission to access HDFS.</li>
-</ul>
-<h3>
-<a id="procedure-1" class="anchor" href="#procedure-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Procedure</h3>
-<p>The following steps are only for Driver Nodes. (Driver nodes are the one which starts the spark context.)</p>
-<ol>
-<li>
-<p><a href="https://github.com/apache/carbondata/blob/master/build/README.md" target=_blank>Build the CarbonData</a> project and get the assembly jar from <code>./assembly/target/scala-2.1x/carbondata_xxx.jar</code> and copy to <code>$SPARK_HOME/carbonlib</code> folder.</p>
-<p><strong>NOTE</strong>: Create the carbonlib folder if it does not exists inside <code>$SPARK_HOME</code> path.</p>
-</li>
-<li>
-<p>Copy the <code>./conf/carbon.properties.template</code> file from CarbonData repository to <code>$SPARK_HOME/conf/</code> folder and rename the file to <code>carbon.properties</code>.</p>
-</li>
-<li>
-<p>Create <code>tar.gz</code> file of carbonlib folder and move it inside the carbonlib folder.</p>
-</li>
-</ol>
-<pre><code>cd $SPARK_HOME
-tar -zcvf carbondata.tar.gz carbonlib/
-mv carbondata.tar.gz carbonlib/
-</code></pre>
-<ol start="4">
-<li>Configure the properties mentioned in the following table in <code>$SPARK_HOME/conf/spark-defaults.conf</code> file.</li>
-</ol>
-<table>
-<thead>
-<tr>
-<th>Property</th>
-<th>Description</th>
-<th>Value</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>spark.master</td>
-<td>Set this value to run the Spark in yarn cluster mode.</td>
-<td>Set yarn-client to run the Spark in yarn cluster mode.</td>
-</tr>
-<tr>
-<td>spark.yarn.dist.files</td>
-<td>Comma-separated list of files to be placed in the working directory of each executor.</td>
-<td><code>$SPARK_HOME/conf/carbon.properties</code></td>
-</tr>
-<tr>
-<td>spark.yarn.dist.archives</td>
-<td>Comma-separated list of archives to be extracted into the working directory of each executor.</td>
-<td><code>$SPARK_HOME/carbonlib/carbondata.tar.gz</code></td>
-</tr>
-<tr>
-<td>spark.executor.extraJavaOptions</td>
-<td>A string of extra JVM options to pass to executors. For instance  <strong>NOTE</strong>: You can enter multiple values separated by space.</td>
-<td><code>-Dcarbon.properties.filepath = carbon.properties</code></td>
-</tr>
-<tr>
-<td>spark.executor.extraClassPath</td>
-<td>Extra classpath entries to prepend to the classpath of executors. <strong>NOTE</strong>: If SPARK_CLASSPATH is defined in spark-env.sh, then comment it and append the values in below parameter spark.driver.extraClassPath</td>
-<td><code>carbondata.tar.gz/carbonlib/*</code></td>
-</tr>
-<tr>
-<td>spark.driver.extraClassPath</td>
-<td>Extra classpath entries to prepend to the classpath of the driver. <strong>NOTE</strong>: If SPARK_CLASSPATH is defined in spark-env.sh, then comment it and append the value in below parameter spark.driver.extraClassPath.</td>
-<td><code>$SPARK_HOME/carbonlib/*</code></td>
-</tr>
-<tr>
-<td>spark.driver.extraJavaOptions</td>
-<td>A string of extra JVM options to pass to the driver. For instance, GC settings or other logging.</td>
-<td><code>-Dcarbon.properties.filepath = $SPARK_HOME/conf/carbon.properties</code></td>
-</tr>
-</tbody>
-</table>
-<ol start="5">
-<li>Add the following properties in <code>$SPARK_HOME/conf/carbon.properties</code>:</li>
-</ol>
-<table>
-<thead>
-<tr>
-<th>Property</th>
-<th>Required</th>
-<th>Description</th>
-<th>Example</th>
-<th>Default Value</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>carbon.storelocation</td>
-<td>NO</td>
-<td>Location where CarbonData will create the store and write the data in its own format. If not specified then it takes spark.sql.warehouse.dir path.</td>
-<td>hdfs://HOSTNAME:PORT/Opt/CarbonStore</td>
-<td>Propose to set HDFS directory</td>
-</tr>
-</tbody>
-</table>
-<ol start="6">
-<li>Verify the installation.</li>
-</ol>
-<pre><code> ./bin/spark-shell --master yarn-client --driver-memory 1g
- --executor-cores 2 --executor-memory 2G
-</code></pre>
-<p><strong>NOTE</strong>: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.</p>
-<p>Getting started with CarbonData : <a href="quick-start-guide.html">Quick Start</a>, <a href="data-management-on-carbondata.html">Data Management on CarbonData</a></p>
-<h2>
-<a id="query-execution-using-carbondata-thrift-server" class="anchor" href="#query-execution-using-carbondata-thrift-server" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Query Execution Using CarbonData Thrift Server</h2>
-<h3>
-<a id="starting-carbondata-thrift-server" class="anchor" href="#starting-carbondata-thrift-server" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Starting CarbonData Thrift Server.</h3>
-<p>a. cd <code>$SPARK_HOME</code></p>
-<p>b. Run the following command to start the CarbonData thrift server.</p>
-<pre><code>./bin/spark-submit
---class org.apache.carbondata.spark.thriftserver.CarbonThriftServer
-$SPARK_HOME/carbonlib/$CARBON_ASSEMBLY_JAR &lt;carbon_store_path&gt;
-</code></pre>
-<table>
-<thead>
-<tr>
-<th>Parameter</th>
-<th>Description</th>
-<th>Example</th>
-</tr>
-</thead>
-<tbody>
-<tr>
-<td>CARBON_ASSEMBLY_JAR</td>
-<td>CarbonData assembly jar name present in the <code>$SPARK_HOME/carbonlib/</code> folder.</td>
-<td>carbondata_2.xx-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar</td>
-</tr>
-<tr>
-<td>carbon_store_path</td>
-<td>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. If not specified then it takes spark.sql.warehouse.dir path.</td>
-<td><code>hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store</code></td>
-</tr>
-</tbody>
-</table>
-<p><strong>NOTE</strong>: From Spark 1.6, by default the Thrift server runs in multi-session mode. Which means each JDBC/ODBC connection owns a copy of their own SQL configuration and temporary function registry. Cached tables are still shared though. If you prefer to run the Thrift server in single-session mode and share all SQL configuration and temporary function registry, please set option <code>spark.sql.hive.thriftServer.singleSession</code> to <code>true</code>. You may either add this option to <code>spark-defaults.conf</code>, or pass it to <code>spark-submit.sh</code> via <code>--conf</code>:</p>
-<pre><code>./bin/spark-submit
---conf spark.sql.hive.thriftServer.singleSession=true
---class org.apache.carbondata.spark.thriftserver.CarbonThriftServer
-$SPARK_HOME/carbonlib/$CARBON_ASSEMBLY_JAR &lt;carbon_store_path&gt;
-</code></pre>
-<p><strong>But</strong> in single-session mode, if one user changes the database from one connection, the database of the other connections will be changed too.</p>
-<p><strong>Examples</strong></p>
-<ul>
-<li>Start with default memory and executors.</li>
-</ul>
-<pre><code>./bin/spark-submit
---class org.apache.carbondata.spark.thriftserver.CarbonThriftServer 
-$SPARK_HOME/carbonlib
-/carbondata_2.xx-x.x.x-SNAPSHOT-shade-hadoop2.7.2.jar
-hdfs://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
-</code></pre>
-<ul>
-<li>Start with Fixed executors and resources.</li>
-</ul>
-<pre><code>./bin/spark-submit
---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://&lt;host_name&gt;:port/user/hive/warehouse/carbon.store
-</code></pre>
-<h3>
-<a id="connecting-to-carbondata-thrift-server-using-beeline" class="anchor" href="#connecting-to-carbondata-thrift-server-using-beeline" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Connecting to CarbonData Thrift Server Using Beeline.</h3>
-<pre><code>     cd $SPARK_HOME
-     ./sbin/start-thriftserver.sh
-     ./bin/beeline -u jdbc:hive2://&lt;thriftserver_host&gt;:port
-
-     Example
-     ./bin/beeline -u jdbc:hive2://10.10.10.10:10000
-</code></pre>
-</div>
-</div>
-</div>
-</div>
-<div class="doc-footer">
-    <a href="#top" class="scroll-top">Top</a>
-</div>
-</div>
-</section>
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/introduction.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/introduction.html b/src/main/webapp/introduction.html
index 29d2aa1..068d711 100644
--- a/src/main/webapp/introduction.html
+++ b/src/main/webapp/introduction.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -228,122 +228,128 @@
 </li>
 </ul>
 <h2>
-<a id="architecture" class="anchor" href="#architecture" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Architecture</h2>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_architecture.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_architecture.png" alt="" style="max-width:100%;"></a></p>
-<h4>
-<a id="spark-interface-layer" class="anchor" href="#spark-interface-layer" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Spark Interface Layer:</h4>
-<p>CarbonData has deep integration with Apache Spark.CarbonData integrates custom Parser,Strategies,Optimization rules into Spark to take advantage of computing performed closer to data.</p>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_spark_integration.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_spark_integration.png" alt="" style="max-width:100%;"></a></p>
-<ol>
+<a id="carbondata-features--functions" class="anchor" href="#carbondata-features--functions" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Features &amp; Functions</h2>
+<p>CarbonData has rich set of featues to support various use cases in Big Data analytics.The below table lists the major features supported by CarbonData.</p>
+<h3>
+<a id="table-management" class="anchor" href="#table-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Table Management</h3>
+<ul>
 <li>
-<strong>Carbon parser</strong> Enhances Spark?s SQL parser to support Carbon specific DDL and DML commands to create carbon table, create aggregate tables, manage data loading, data retention and cleanup.</li>
+<h5>
+<a id="ddl-create-alterdropctas" class="anchor" href="#ddl-create-alterdropctas" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DDL (Create, Alter,Drop,CTAS)</h5>
+</li>
+</ul>
+<p>?	CarbonData provides its own DDL to create and manage carbondata tables.These DDL conform to 			Hive,Spark SQL format and support additional properties and configuration to take advantages of CarbonData functionalities.</p>
+<ul>
 <li>
-<strong>Carbon Strategies</strong>:- Modify Spark SQL?s physical query execution plan to push down possible operations to Carbon for example:- Grouping, Distinct Count, Top N etc.. for improving query performance.</li>
+<h5>
+<a id="dmlloadinsert" class="anchor" href="#dmlloadinsert" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DML(Load,Insert)</h5>
+<p>CarbonData provides its own DML to manage data in carbondata tables.It adds many customizations through configurations to completely customize the behavior as per user requirement scenarios.</p>
+</li>
 <li>
-<strong>Carbon Data RDD</strong>:- Makes the data present in Carbon tables visible to Spark as a RDD which enables spark to perform distributed computation on Carbon tables.</li>
-</ol>
-<h4>
-<a id="carbon-processor" class="anchor" href="#carbon-processor" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Carbon Processor:</h4>
-<p>Receives a query execution fragment from spark and executes the same on the Carbon storage. This involves Scanning the carbon store files for matching record, using the indices to directly locate the row sets and even the rows that may containing the data being searched for. The Carbon processor also performs all pushed down operations such as</p>
-<p>Aggregation/Group By</p>
-<p>Distinct Count</p>
-<p>Top N</p>
-<p>Expression Evaluation</p>
-<p>And many more?</p>
-<h4>
-<a id="carbon-storage" class="anchor" href="#carbon-storage" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Carbon Storage:</h4>
-<p>Custom columnar data store which is heavily compressed, binary, dictionary encoded and heavily indexed.Usaually stored in HDFS.</p>
-<h2>
-<a id="carbondata-features" class="anchor" href="#carbondata-features" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Features</h2>
-<p>CarbonData has rich set of featues to support various use cases in Big Data analytics.</p>
-<h2>
-<a id="design" class="anchor" href="#design" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Design</h2>
-<ul>
+<h5>
+<a id="update-and-delete" class="anchor" href="#update-and-delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Update and Delete</h5>
+<p>CarbonData supports Update and Delete on Big Data.CarbonData provides the syntax similar to Hive to support IUD operations on CarbonData tables.</p>
+</li>
 <li>
-<h3>
-<a id="dictionary-encoding" class="anchor" href="#dictionary-encoding" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Dictionary Encoding</h3>
+<h5>
+<a id="segment-management" class="anchor" href="#segment-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Segment Management</h5>
+<p>CarbonData has unique concept of segments to manage incremental loads to CarbonData tables effectively.Segment management helps to easily control the table, perform easy retention, and is also used to provide transaction capability for operations being performed.</p>
+</li>
+<li>
+<h5>
+<a id="partition" class="anchor" href="#partition" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Partition</h5>
+<p>CarbonData supports 2 kinds of partitions.1.partition similar to hive partition.2.CarbonData partition supporting hash,list,range partitioning.</p>
+</li>
+<li>
+<h5>
+<a id="compaction" class="anchor" href="#compaction" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Compaction</h5>
+<p>CarbonData manages incremental loads as segments.Compaction help to compact the growing number of segments and also to improve query filter pruning.</p>
+</li>
+<li>
+<h5>
+<a id="external-tables" class="anchor" href="#external-tables" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>External Tables</h5>
+<p>CarbonData can read any carbondata file and automatically infer schema from the file and provide a relational table view to perform sql queries using Spark or any other applicaion.</p>
 </li>
 </ul>
-<p>CarbonData supports encoding of data with suggogate values to reduce storage space and speed up processing.Most databases and big data SQL data stores adopt dictionary encoding(integer surrogate numbers) to achieve data compression.Unlike other column store databases where the dictionary is local to each data block, CarbonData maintains a global dictionary which provides opportunity for lazy conversion to actual values enabling all computation to be performed on the lightweight surrogate values.</p>
+<h3>
+<a id="datamaps" class="anchor" href="#datamaps" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DataMaps</h3>
+<ul>
+<li>
 <h5>
-<a id="dictionary-generation" class="anchor" href="#dictionary-generation" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Dictionary generation</h5>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_dict_encoding.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_dict_encoding.png" alt="" style="max-width:100%;"></a></p>
+<a id="pre-aggregate" class="anchor" href="#pre-aggregate" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Pre-Aggregate</h5>
+<p>CarbonData has concept of datamaps to assist in pruning of data while querying so that performance is faster.Pre Aggregate tables are kind of datamaps which can improve the query performance by order of magnitude.CarbonData will automatically pre-aggregae the incremental data and re-write the query to automatically fetch from the most appropriate pre-aggregate table to serve the query faster.</p>
+</li>
+<li>
 <h5>
-<a id=.htmlk-indexing" class="anchor" href=".htmlk-indexing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>MDK Indexing</h5>
-<p>All the surrogate keys are byte packed to generate an MDK (Multi Dimensional Key) Index.</p>
-<p>Any non surrogate columns of String data types are compressed using one of the configured compression algorithms and stored.For those numeric columns where surrogates are not generated, such data is stored as it is after compression.</p>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk.png" alt="image-20180903212418381" style="max-width:100%;"></a></p>
+<a id="time-series" class="anchor" href="#time-series" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Time Series</h5>
+<p>CarbonData has built in understanding of time order(Year, month,day,hour, minute,second).Time series is a pre-aggregate table which can automatically roll-up the data to the desired level during incremental load and serve the query from the most appropriate pre-aggregate table.</p>
+</li>
+<li>
+<h5>
+<a id="bloom-filter" class="anchor" href="#bloom-filter" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Bloom filter</h5>
+<p>CarbonData supports bloom filter as a datamap in order to quickly and efficiently prune the data for scanning and acheive faster query performance.</p>
+</li>
+<li>
 <h5>
-<a id="sorted.htmlk" class="anchor" href="#sorted.htmlk" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Sorted MDK</h5>
-<p>The data is sorted based on the MDK Index.Sorting helps for logical grouping of similar data and there by aids in faster look up during query.</p>
-<h4>
-<a id="" class="anchor" href="#" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk_sort.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata.htmlk_sort.png" alt="image-20180903212525214" style="max-width:100%;"></a>
-</h4>
+<a id="lucene" class="anchor" href="#lucene" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Lucene</h5>
+<p>Lucene is popular for indexing text data which are long.CarbonData provides a lucene datamap so that text columns can be indexed using lucene and use the index result for efficient pruning of data to be retrieved during query.</p>
+</li>
+<li>
 <h5>
-<a id="custom-columnar-encoding" class="anchor" href="#custom-columnar-encoding" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Custom Columnar Encoding</h5>
-<p>The Sorted MDK Index is split into each column.Unlike other stores where the column is compressed and stored as it is, CarbonData sorts this column data so that Binary Search can be performed on individual column data based on the filter conditions.This aids in magnitude increase in query performance and also in better compression.Since the individual column's data gets sorted, it is necessary to maintain the row mapping with the sorted MDK Index data in order to retrieve data from other columns which are not participating in filter.This row mapping is termed as <strong>Inverted Index</strong> and is stored along with the column data.The below picture depicts the logical column view.User has the option to <strong>turn off</strong> Inverted Index for such columns where filters are never applied or is very rare.In such cases, scanning would be sequential, but can aid in reducing the storage size(occupied due to inverted index data).</p>
-<h4>
-<a id="-1" class="anchor" href="#-1" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_blocklet_view.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_blocklet_view.png" alt="" style="max-width:100%;"></a>
-</h4>
+<a id="mv-materialized-views" class="anchor" href="#mv-materialized-views" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>MV (Materialized Views)</h5>
+<p>MVs are kind of pre-aggregate tables which can support efficent query re-write and processing.CarbonData provides MV which can rewrite query to fetch from any table(including non-carbondata tables).Typical usecase is to store the aggregated data of a non-carbondata fact table into carbondata and use mv to rewrite the query to fetch from carbondata.</p>
+</li>
+</ul>
+<h3>
+<a id="streaming" class="anchor" href="#streaming" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Streaming</h3>
 <ul>
 <li>
+<h5>
+<a id="spark-streaming" class="anchor" href="#spark-streaming" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Spark Streaming</h5>
+<p>CarbonData supports streaming of data into carbondata in near-realtime and make it immediately available for query.CarbonData provides a DSL to create source and sink tables easily without the need for the user to write his application.</p>
+</li>
+</ul>
 <h3>
-<a id="carbondata-storage-format" class="anchor" href="#carbondata-storage-format" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Storage Format</h3>
-<p>CarbonData has a unique storage structure which aids in efficient storage and retrieval of data.Please refer to <a href="#./file-structure-of-carbondata.html">File Structure of CarbonData</a> for detailed information on the format.</p>
+<a id="sdk" class="anchor" href="#sdk" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>SDK</h3>
+<ul>
+<li>
+<h5>
+<a id="carbondata-writer" class="anchor" href="#carbondata-writer" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData writer</h5>
+<p>CarbonData supports writing data from non-spark application using SDK.Users can use SDK to generate carbondata files from custom applications.Typical usecase is to write the streaming application plugged in to kafka and use carbondata as sink(target) table for storing.</p>
 </li>
 <li>
-<h3>
-<a id="indexing" class="anchor" href="#indexing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Indexing</h3>
-<p>CarbonData maintains multiple indexes at multiple levels to assist in efficient pruning of unwanted data from scan during query.Also CarbonData has support for plugging in external indexing solutions to speed up the query process.</p>
 <h5>
-<a id="min-max-indexing" class="anchor" href="#min-max-indexing" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Min-Max Indexing</h5>
-<p>Storing data along with index significantly accelerates query performance and reduces the I/O scans and CPU resources in case of filters in the query. CarbonData index consists of multiple levels of indices, a processing framework can leverage this index to reduce the number of tasks it needs to schedule and process. It can also do skip scan in more fine grained units (called blocklet) in task side scanning instead of scanning the whole file.  <strong>CarbonData maintains Min-Max Index for all the columns.</strong></p>
-<p>CarbonData maintains a separate index file which contains the footer information for efficient IO reads.</p>
-<p>Using the Min-Max info in these index files, two levels of filtering can be achieved.</p>
-<p>Min-Max at the carbondata file level,to efficiently prune the files when the filter condition doesn't fall in the range.This information when maintained at the Spark Driver, will help to efficiently schedule the tasks for scanning</p>
-<p>Min-Max at the blocklet level, to efficiently prune the blocklets when the filter condition doesn't fall in the range.This information when maintained at the executor can significantly reduce the amount unnecessary data processed by the executor tasks.</p>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-minmax-blocklet.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-minmax-blocklet.png" alt="" style="max-width:100%;"></a></p>
+<a id="carbondata-reader" class="anchor" href="#carbondata-reader" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData reader</h5>
+<p>CarbonData supports reading of data from non-spark application using SDK.Users can use the SDK to read the carbondata files from their application and do custom processing.</p>
 </li>
+</ul>
+<h3>
+<a id="storage" class="anchor" href="#storage" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Storage</h3>
+<ul>
 <li>
-<h4>
-<a id="datamaps" class="anchor" href="#datamaps" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DataMaps</h4>
-<p>DataMap is a framework for indexing and also for statistics that can be used to add primary index (Blocklet Index) , secondary index type and statistical type to CarbonData.</p>
-<p>DataMap is a standardized general interface which CarbonData uses to prune data blocks for scanning.</p>
-<p>DataMaps are of 2 types:</p>
-<p><strong>CG(Coarse Grained) DataMaps</strong> Can prune data to the blocklet or to Page level.ie., Holds information for deciding which blocks/blocklets to be scanned.This DataMap is used in Spark Driver to decide the number of tasks to be scheduled.</p>
-<p><strong>FG(Fine Grained) DataMaps</strong> Can prune data to row level.This DataMap is used in Spark executor for scanning an fetching the data much faster.</p>
-<p>Since DataMap interfaces are generalised, We can write a thin adaptor called as <strong>DataMap Providers</strong> to interface between CarbonData and other external Indexing engines. For eg., Lucene, Solr,ES,...</p>
-<p>CarbonData has its own DSL to create and manage DataMaps.Please refer to <a href="#./datamap/datamap-management.html#overview">CarbonData DSL</a> for more information.</p>
-<p>The below diagram explains about the DataMap execution in CarbonData.</p>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-datamap.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata-datamap.png" alt="" style="max-width:100%;"></a></p>
+<h5>
+<a id="s3" class="anchor" href="#s3" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>S3</h5>
+<p>CarbonData can write to S3, OBS or any cloud storage confirming to S3 protocol.CarbonData uses the HDFS api to write to cloud object stores.</p>
 </li>
 <li>
-<h4>
-<a id="update--delete" class="anchor" href="#update--delete" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Update &amp; Delete</h4>
+<h5>
+<a id="hdfs" class="anchor" href="#hdfs" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>HDFS</h5>
+<p>CarbonData uses HDFS api to write and read data from HDFS.CarbonData can take advantage of the locality information to efficiently suggest spark to run tasks near to the data.</p>
 </li>
 </ul>
-<p>CarbonData supports Update and delete operations over big data.This functionality is not targetted for OLTP scenarios where high concurrent update/delete are required.Following are the assumptions considered when this feature is designed.</p>
-<ol>
-<li>Updates or Deletes are periodic and in Bulk</li>
-<li>Updates or Deletes are atomic</li>
-<li>Data is immediately visible</li>
-<li>Concurrent query to be allowed during an update or delete operation</li>
-<li>Single statement auto-commit support (not OLTP-style transaction)</li>
-</ol>
-<p>Since data stored in HDFS are immutable,data blocks cannot be updated in-place.Re-write of entire data block is not efficient for IO and also is a slow process.</p>
-<p>To over come these limitations, CarbonData adopts methodology of writing a delta file containing the rows to be deleted and another delta file containing the values to be updated with.During processing, These two delta files are merged with the main carbondata file and the correct result is returned for the query.</p>
-<p>The below diagram describes the process.</p>
-<p><a href="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_update_delete.png" target="_blank" rel="noopener noreferrer"><img src="/Users/aditi_advith/Documents/code/carbondata/docs/images/carbondata_update_delete.png" alt="" style="max-width:100%;"></a></p>
 <h2>
 <a id="integration-with-big-data-ecosystem" class="anchor" href="#integration-with-big-data-ecosystem" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Integration with Big Data ecosystem</h2>
-<p>Refer to Integration with <a href="#./quick-start-guide.html#spark">Spark</a>, <a href="#./quick-start-guide.html#presto">Presto</a> for detailed information on integrating CarbonData with these execution engines.</p>
+<p>Refer to Integration with <a href="./quick-start-guide.html#spark">Spark</a>, <a href="./quick-start-guide.html#presto">Presto</a> for detailed information on integrating CarbonData with these execution engines.</p>
 <h2>
 <a id="scenarios-where-carbondata-is-suitable" class="anchor" href="#scenarios-where-carbondata-is-suitable" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Scenarios where CarbonData is suitable</h2>
+<p>CarbonData is useful in various analytical work loads.Some of the most typical usecases where CarbonData is being used is <a href="./usecases.html">documented here</a>.</p>
+<h2>
+<a id="performance-results" class="anchor" href="#performance-results" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Performance Results</h2>
+<p><a href="../docs/images/carbondata-performance.png?raw=true" target="_blank" rel="noopener noreferrer"><img src="https://github.com/apache/carbondata/blob/master/docs/images/carbondata-performance.png?raw=true" alt="Performance Results" style="max-width:100%;"></a></p>
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__intro').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -358,4 +364,4 @@ $(function() { $('.b-nav__intro').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/language-manual.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/language-manual.html b/src/main/webapp/language-manual.html
index 54c29ac..a0ea674 100644
--- a/src/main/webapp/language-manual.html
+++ b/src/main/webapp/language-manual.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -220,7 +220,6 @@
 <li>
 <a href="./ddl-of-carbondata.html">DDL:</a><a href="./ddl-of-carbondata.html#create-table">Create</a>,<a href="./ddl-of-carbondata.html#drop-table">Drop</a>,<a href="./ddl-of-carbondata.html#partition">Partition</a>,<a href="./ddl-of-carbondata.html#bucketing">Bucketing</a>,<a href="./ddl-of-carbondata.html#alter-table">Alter</a>,<a href="./ddl-of-carbondata.html#create-table-as-select">CTAS</a>,<a href="./ddl-of-carbondata.html#create-external-table">External Table</a>
 </li>
-<li>Indexes</li>
 <li>
 <a href="./datamap-management.html">DataMaps</a>
 <ul>
@@ -254,8 +253,7 @@ $(function() {
     $('.b-nav__docs').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -270,4 +268,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/lucene-datamap-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/lucene-datamap-guide.html b/src/main/webapp/lucene-datamap-guide.html
index 038cb9d..b8164a2 100644
--- a/src/main/webapp/lucene-datamap-guide.html
+++ b/src/main/webapp/lucene-datamap-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -248,7 +248,7 @@ search tokenized word or pattern of it using lucene query on text content.</p>
   age int,
   city string,
   country string)
-STORED BY 'carbondata'
+STORED AS carbondata
 </code></pre>
 <p>User can create Lucene datamap using the Create DataMap DDL:</p>
 <pre><code>CREATE DATAMAP dm
@@ -328,7 +328,7 @@ select * from datamap_test where TEXT_MATCH('name:*n*')
 
 select * from datamap_test where TEXT_MATCH('name:*10 -name:*n*')
 </code></pre>
-<p><strong>Note:</strong> For lucene queries and syntax, refer to <a href="www.lucenetutorial.com/lucene-query-syntax.html">lucene-syntax</a></p>
+<p><strong>Note:</strong> For lucene queries and syntax, refer to <a href="http://www.lucenetutorial.com/lucene-query-syntax.html" target=_blank rel="nofollow">lucene-syntax</a></p>
 <h2>
 <a id="data-management-with-lucene-datamap" class="anchor" href="#data-management-with-lucene-datamap" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Management with lucene datamap</h2>
 <p>Once there is lucene datamap is created on the main table, following command on the main
@@ -363,8 +363,7 @@ $(function() {
     $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -379,4 +378,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/mainpage.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/mainpage.html b/src/main/webapp/mainpage.html
deleted file mode 100644
index d515853..0000000
--- a/src/main/webapp/mainpage.html
+++ /dev/null
@@ -1,214 +0,0 @@
-<!DOCTYPE html>
-<html lang="en">
-<head>
-    <meta charset="utf-8">
-    <meta http-equiv="X-UA-Compatible" content="IE=edge">
-    <meta name="viewport" content="width=device-width, initial-scale=1">
-    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
-    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
-    <title>CarbonData</title>
-    <style>
-
-    </style>
-    <!-- Bootstrap -->
-
-    <link rel="stylesheet" href="css/bootstrap.min.css">
-    <link href="css/style.css" rel="stylesheet">
-    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
-    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
-    <!--[if lt IE 9]>
-    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
-    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
-    <![endif]-->
-    <script src="js/jquery.min.js"></script>
-    <script src="js/bootstrap.min.js"></script>
-
-
-</head>
-<body>
-<header>
-    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
-        <div class="container">
-            <div class="navbar-header">
-                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
-                        class="navbar-toggle collapsed" type="button">
-                    <span class="sr-only">Toggle navigation</span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                    <span class="icon-bar"></span>
-                </button>
-                <a href="index.html" class="logo">
-                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
-                </a>
-            </div>
-            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
-                <ul class="nav navbar-nav navbar-right navlist-custom">
-                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
-                    </li>
-                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false"> Download <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.1/"
-                                   target="_blank">Apache CarbonData 1.4.1</a></li>
-							<li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
-                                   target="_blank">Apache CarbonData 1.4.0</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
-                                   target="_blank">Apache CarbonData 1.3.1</a></li>
-                            <li>
-                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
-                                   target="_blank">Apache CarbonData 1.3.0</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
-                                   target="_blank">Release Archive</a></li>
-                        </ul>
-                    </li>
-                    <li><a href="mainpage.html" class="active">Documentation</a></li>
-                    <li class="dropdown">
-                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
-                           aria-expanded="false">Community <span class="caret"></span></a>
-                        <ul class="dropdown-menu">
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
-                                   target="_blank">Contributing to CarbonData</a></li>
-                            <li>
-                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
-                                   target="_blank">Release Guide</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
-                                   target="_blank">Project PMC and Committers</a></li>
-                            <li>
-                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
-                                   target="_blank">CarbonData Meetups</a></li>
-                            <li><a href="security.html">Apache CarbonData Security</a></li>
-                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
-                                Jira</a></li>
-                            <li><a href="videogallery.html">CarbonData Videos </a></li>
-                        </ul>
-                    </li>
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li class="dropdown">
-                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
-                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
-                        <ul class="dropdown-menu">
-                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
-                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
-                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
-                                   target="_blank">Sponsorship</a></li>
-                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
-                        </ul>
-                    </li>
-
-                    <li>
-                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
-
-                    </li>
-
-                </ul>
-            </div><!--/.nav-collapse -->
-            <div id="search-box">
-                <form method="get" action="http://www.google.com/search">
-                    <div class="search-block">
-                        <table border="0" cellpadding="0" width="100%">
-                            <tr>
-                                <td style="width:80%">
-                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
-                                           class="search-input" placeholder="Search...."    required/>
-                                </td>
-                                <td style="width:20%">
-                                    <input type="submit" value="Search"/></td>
-                            </tr>
-                            <tr>
-                                <td align="left" style="font-size:75%" colspan="2">
-                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
-                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
-                                </td>
-                            </tr>
-                        </table>
-                    </div>
-                </form>
-            </div>
-        </div>
-    </nav>
-</header> <!-- end Header part -->
-
-<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
-
-<section><!-- Dashboard nav -->
-    <div class="container-fluid q">
-        <div class="col-sm-12  col-md-12 maindashboard">
-            <div class="row">
-                <section>
-                    <div style="padding:10px 15px;">
-                        <div id="viewpage" name="viewpage">
-                            <div class="doc-heading">
-                                <h4 class="title">Documentation
-                                    <span class="title-underline"></span>
-                                </h4>
-                            </div>
-
-                            <div class="row">
-
-                                <div class="col-sm-12  col-md-12">
-                                    <span class="text-justify">
-                                        Welcome to Apache CarbonData. Apache CarbonData is a new big data file format for faster interactive query using advanced columnar storage, index, compression and encoding techniques to improve computing efficiency, which helps in speeding up queries by an order of magnitude faster over PetaBytes of data. This user guide provides a detailed description about the CarbonData and its features.
-                                        Let's get started !
-                                    </span>
-                                    <hr style="margin: 12px 0 8px">
-                                    <div>
-                                        <ul class="sub-nav">
-                                            <li><a href="quick-start-guide.html">Quick Start</a></li>
-                                            <li><a href="file-structure-of-carbondata.html">CarbonData File Structure</a></li>
-                                            <li><a href="supported-data-types-in-carbondata.html">Data Types</a></li>
-                                            <li><a href="data-management-on-carbondata.html">Data Management On CarbonData</a></li>
-                                            <li><a href="installation-guide.html">Installation Guide</a></li>
-                                            <li><a href="configuration-parameters.html">Configuring CarbonData</a></li>
-                                            <li><a href="streaming-guide.html">Streaming Guide</a></li>
-                                            <li><a href="sdk-guide.html">SDK Guide</a></li>
-											<li><a href="s3-guide.html">S3 Guide (Alpha Feature)</a></li>
-                                            <li><a href="datamap-developer-guide.html">DataMap Developer Guide</a></li>
-											<li><a href="datamap-management.html">CarbonData DataMap Management</a></li>
-                                            <li><a href="bloomfilter-datamap-guide.html">CarbonData BloomFilter DataMap (Alpha Feature)</a></li>
-                                            <li><a href="lucene-datamap-guide.html">CarbonData Lucene DataMap (Alpha Feature)</a></li>
-                                            <li><a href="preaggregate-datamap-guide.html">CarbonData Pre-aggregate DataMap</a></li>
-                                            <li><a href="timeseries-datamap-guide.html">CarbonData Timeseries DataMap</a></li>
-                                            <li><a href="faq.html">FAQs</a></li>
-                                            <li><a href="troubleshooting.html">Troubleshooting</a></li>
-                                            <li><a href="useful-tips-on-carbondata.html">Useful Tips</a></li>
-
-                                        </ul>
-                                    </div>
-                                </div>
-                            </div>
-                        </div>
-                        <div class="doc-footer">
-                            <a href="#top" class="scroll-top">Top</a>
-                        </div>
-                    </div>
-                </section>
-            </div>
-        </div>
-    </div>
-</section><!-- End systemblock part -->
-</div>
-</div>
-</div>
-</section><!-- End systemblock part -->
-<script src="js/custom.js"></script>
-</body>
-</html>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/performance-tuning.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/performance-tuning.html b/src/main/webapp/performance-tuning.html
index 49b3d3a..480911c 100644
--- a/src/main/webapp/performance-tuning.html
+++ b/src/main/webapp/performance-tuning.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -218,6 +218,7 @@ The following sections will elaborate on the below topics :</p>
 <li><a href="#suggestions-to-create-carbondata-table">Suggestions to create CarbonData Table</a></li>
 <li><a href="#configuration-for-optimizing-data-loading-performance-for-massive-data">Configuration for Optimizing Data Loading performance for Massive Data</a></li>
 <li><a href="#configurations-for-optimizing-carbondata-performance">Optimizing Query Performance</a></li>
+<li><a href="#compaction-configurations-for-optimizing-carbondata-query-performance">Compaction Configurations for Optimizing CarbonData Query Performance</a></li>
 </ul>
 <h2>
 <a id="suggestions-to-create-carbondata-table" class="anchor" href="#suggestions-to-create-carbondata-table" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Suggestions to Create CarbonData Table</h2>
@@ -299,7 +300,7 @@ The create table command can be modified as suggested below :</p>
   counter_1, Decimal
   ...
   
-  )STORED BY 'carbondata'
+  )STORED AS carbondata
   TBLPROPERTIES ('SORT_COLUMNS'='msisdn, Dime_1')
 </code></pre>
 <p>Now the query with MSISDN in the filter will be more efficient.</p>
@@ -320,7 +321,7 @@ The create table command can be modified as suggested below :</p>
     counter_1, Decimal
     ...
     
-    )STORED BY 'carbondata'
+    )STORED AS carbondata
     TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
 </code></pre>
 <ul>
@@ -337,7 +338,7 @@ The create table command can be modified as below :</p>
     counter_1 decimal,
     counter_2 double,
     ...
-    )STORED BY 'carbondata'
+    )STORED AS carbondata
     TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
 </code></pre>
 <p>The result of performance analysis of test-case shows reduction in query execution time from 15 to 3 seconds, thereby improving performance by nearly 5 times.</p>
@@ -356,12 +357,12 @@ Incremental values are efficient in using min/max index. The create table comman
   END_TIME bigint,
   ...
   counter_100 double
-  )STORED BY 'carbondata'
+  )STORED AS carbondata
   TBLPROPERTIES ('SORT_COLUMNS'='Dime_1, HOST, MSISDN')
 </code></pre>
 <p><strong>NOTE:</strong></p>
 <ul>
-<li>BloomFilter can be created to enhance performance for queries with precise equal/in conditions. You can find more information about it in BloomFilter datamap <a href="https://github.com/apache/carbondata/blob/master/docs/datamap/bloomfilter-datamap-guide.html" target=_blank>document</a>.</li>
+<li>BloomFilter can be created to enhance performance for queries with precise equal/in conditions. You can find more information about it in BloomFilter datamap <a href="./bloomfilter-datamap-guide.html">document</a>.</li>
 </ul>
 <h2>
 <a id="configuration-for-optimizing-data-loading-performance-for-massive-data" class="anchor" href="#configuration-for-optimizing-data-loading-performance-for-massive-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Configuration for Optimizing Data Loading performance for Massive Data</h2>
@@ -507,11 +508,72 @@ scenarios. After the completion of POC, some of the configurations impacting the
 </tbody>
 </table>
 <p>Note: If your CarbonData instance is provided only for query, you may specify the property 'spark.speculation=true' which is in conf directory of spark.</p>
+<h2>
+<a id="compaction-configurations-for-optimizing-carbondata-query-performance" class="anchor" href="#compaction-configurations-for-optimizing-carbondata-query-performance" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Compaction Configurations for Optimizing CarbonData Query Performance</h2>
+<p>CarbonData provides many configurations to tune the compaction behavior so that query peformance is improved.</p>
+<p>Based on the number of cores available in the node, it is recommended to tune the configuration 	<em><strong>carbon.number.of.cores.while.compacting</strong></em> appropriately.Configuring a higher value will improve the overall compaction performance.</p>
+<p> </p>
+<table>
+<tbody>
+<tr>
+<td>No</td>
+<td> Data Loading frequency</td>
+<td>Data Size of each load</td>
+<td>Minor Compaction configuration</td>
+<td> Major compaction configuration</td>
+</tr>
+<tr>
+<td>1</td>
+<td> Batch(Once is several Hours)</td>
+<td>Big</td>
+<td> Not Suggested</td>
+<td>Configure Major Compaction size of 3-4 load size.Perform Major compaction once in a day</td>
+</tr>
+<tr>
+<td rowspan="2">2</td>
+<td rowspan="2"> Batch(Once in few minutes) </td>
+<td>Big </td>
+<td>
+<p> Minor compaction (2,2).</p>
+<p>Enable Auto compaction, if high rate data loading speed is not required or the time between loads is sufficient to run the compaction</p>
+</td>
+<td>Major compaction size of 10 load size.Perform Major compaction once in a day</td>
+</tr>
+<tr>
+<td>Small</td>
+<td>
+<p>Minor compaction (6,6).</p>
+<p>Enable Auto compaction, if high rate data loading speed is not required or the time between loads is sufficient to run the compaction</p>
+</td>
+<td>Major compaction size of 10 load size.Perform Major compaction once in a day</td>
+</tr>
+<tr>
+<td>3</td>
+<td> History data loaded as single load,incremental loads matches (1) or (2)</td>
+<td>Big</td>
+<td>
+<p> Configure ALLOWED_COMPACTION_DAYS to exclude the History load.</p>
+<p>Configure Minor compaction configuration based condition (1) or (2)</p>
+</td>
+<td> Configure Major compaction size smaller than the history load size.</td>
+</tr>
+<tr>
+<td>4</td>
+<td> There can be error in recent data loaded.Need reload sometimes</td>
+<td> (1) or (2)</td>
+<td>
+<p> Configure COMPACTION_PRESERVE_SEGMENTS</p>
+<p>to exclude the recent few segments from compacting.</p>
+<p>Configure Minor compaction configuration based condition (1) or (2)</p>
+</td>
+<td>Same as (1) or (2) </td>
+</tr>
+</tbody>
+</table>
 <script>
 // Show selected style on nav item
 $(function() { $('.b-nav__perf').addClass('selected'); });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -526,4 +588,4 @@ $(function() { $('.b-nav__perf').addClass('selected'); });
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/a51dc596/src/main/webapp/preaggregate-datamap-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/preaggregate-datamap-guide.html b/src/main/webapp/preaggregate-datamap-guide.html
index 9220c84..6b0783e 100644
--- a/src/main/webapp/preaggregate-datamap-guide.html
+++ b/src/main/webapp/preaggregate-datamap-guide.html
@@ -159,7 +159,7 @@
                         <div class="nav__inner">
                             <a class="b-nav__intro nav__item" href="./introduction.html">introduction</a>
                             <a class="b-nav__quickstart nav__item" href="./quick-start-guide.html">quick start</a>
-                            <a class="b-nav__uses nav__item" href="./usescases.html">use cases</a>
+                            <a class="b-nav__uses nav__item" href="./usecases.html">use cases</a>
 
                             <div class="nav__item nav__item__with__subs">
                                 <a class="b-nav__docs nav__item nav__sub__anchor" href="./language-manual.html">Language Reference</a>
@@ -179,9 +179,9 @@
                                 <a class="nav__item nav__sub__item" href="./timeseries-datamap-guide.html">Time Series</a>
                             </div>
 
-                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Support</a>
                             <a class="b-nav__api nav__item" href="./sdk-guide.html">API</a>
                             <a class="b-nav__perf nav__item" href="./performance-tuning.html">Performance Tuning</a>
+                            <a class="b-nav__s3 nav__item" href="./s3-guide.html">S3 Storage</a>
                             <a class="b-nav__faq nav__item" href="./faq.html">FAQ</a>
                             <a class="b-nav__contri nav__item" href="./how-to-contribute-to-apache-carbondata.html">Contribute</a>
                             <a class="b-nav__security nav__item" href="./security.html">Security</a>
@@ -194,9 +194,9 @@
                         <div class="b-nav__uses navindicator__item"></div>
                         <div class="b-nav__docs navindicator__item"></div>
                         <div class="b-nav__datamap navindicator__item"></div>
-                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__api navindicator__item"></div>
                         <div class="b-nav__perf navindicator__item"></div>
+                        <div class="b-nav__s3 navindicator__item"></div>
                         <div class="b-nav__faq navindicator__item"></div>
                         <div class="b-nav__contri navindicator__item"></div>
                         <div class="b-nav__security navindicator__item"></div>
@@ -256,7 +256,7 @@
 <span class="pl-s">      | country string,</span>
 <span class="pl-s">      | quantity int,</span>
 <span class="pl-s">      | price bigint)</span>
-<span class="pl-s">      | STORED BY 'carbondata'</span>
+<span class="pl-s">      | STORED AS carbondata</span>
 <span class="pl-s">    <span class="pl-pds">"""</span></span>.stripMargin)
  
  <span class="pl-c"><span class="pl-c">//</span> Create pre-aggregate table on the main table</span>
@@ -347,7 +347,7 @@ in production SQLs.</p>
   country string,
   quantity int,
   price bigint)
-STORED BY 'carbondata'
+STORED AS carbondata
 </code></pre>
 <p>User can create pre-aggregate tables using the Create DataMap DDL</p>
 <pre><code>CREATE DATAMAP agg_sales
@@ -476,8 +476,7 @@ $(function() {
     $('.b-nav__datamap').parent().toggleClass('nav__item__with__subs--expanded');
   }
 });
-</script>
-</div>
+</script></div>
 </div>
 </div>
 </div>
@@ -492,4 +491,4 @@ $(function() {
 </section><!-- End systemblock part -->
 <script src="js/custom.js"></script>
 </body>
-</html>
+</html>
\ No newline at end of file