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 2017/05/24 04:56:57 UTC

[1/9] carbondata-site git commit: added markdowns

Repository: carbondata-site
Updated Branches:
  refs/heads/asf-site cf9ebc6d3 -> 20e101ec6


http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/7a262a09/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
new file mode 100644
index 0000000..b1ff903
--- /dev/null
+++ b/src/site/markdown/useful-tips-on-carbondata.md
@@ -0,0 +1,180 @@
+<!--
+    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 above topics :
+
+* [Suggestions to create CarbonData Table](#suggestions-to-create-carbondata-table)
+* [Configurations For Optimizing CarbonData Performance](#configurations-for-optimizing-carbondata-performance)
+
+## Suggestions to Create CarbonData Table
+
+Recently CarbonData was used to analyze performance of Telecommunication field.
+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   | Numeric(20,0) | NA          | Measure     |
+| ...         | ...           | NA          | Measure     |
+| counter_100 | Numeric(20,0) | NA          | Measure     |
+
+CarbonData has more than 50 test cases, on the basis of these we have following suggestions to enhance the query performance :
+
+
+
+* **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,
+  ...
+  )STORED BY 'org.apache.carbondata.format' 
+  TBLPROPERTIES ( 'DICTIONARY_EXCLUDE'='MSISDN,..',
+  'DICTIONARY_INCLUDE'='...');
+```
+  
+  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 as Dime_1 has the lowest cardinality. 
+  The create table command can be modified as suggested below :
+
+```
+  create table carbondata_table(
+  Dime_1 String,
+  HOST String,
+  MSISDN String,
+  ...
+  )STORED BY 'org.apache.carbondata.format' 
+  TBLPROPERTIES ( 'DICTIONARY_EXCLUDE'='MSISDN,HOST..',
+  'DICTIONARY_INCLUDE'='Dime_1..');
+```
+
+
+* **Put the Dimension type columns in order of low to high cardinality**
+
+  If the columns used to filter are not frequently used, then it is suggested to order all the columns of dimension type in order of low to high cardinality.
+The create table command can be modified as below :
+
+```
+  create table carbondata_table(
+  Dime_1 String,
+  BEGIN_TIME bigint
+  HOST String,
+  MSISDN String,
+  ...
+  )STORED BY 'org.apache.carbondata.format' 
+  TBLPROPERTIES ( 'DICTIONARY_EXCLUDE'='MSISDN,HOST,IMSI..',
+  'DICTIONARY_INCLUDE'='Dime_1,END_TIME,BEGIN_TIME..');
+```
+
+
+* **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
+  HOST String,
+  MSISDN String,
+  counter_1 double,
+  counter_2 double,
+  ...
+  counter_100 double
+  )STORED BY 'org.apache.carbondata.format' 
+  TBLPROPERTIES ( 'DICTIONARY_EXCLUDE'='MSISDN,HOST,IMSI',
+  'DICTIONARY_INCLUDE'='Dime_1,END_TIME,BEGIN_TIME');
+```
+  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 start_time is incremental for each load, it is
+suggested to put start_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,
+  ...
+  counter_100 double
+  )STORED BY 'org.apache.carbondata.format' 
+  TBLPROPERTIES ( 'DICTIONARY_EXCLUDE'='MSISDN,HOST,IMSI',
+  'DICTIONARY_INCLUDE'='Dime_1,END_TIME,BEGIN_TIME'); 
+```
+
+
+* **Avoid adding high cardinality columns to dictionary**
+
+  If the system has low memory configuration, then it is suggested to exclude high cardinality columns from the dictionary to 
+enhance load performance. Creation of  dictionary for high cardinality columns at time of load will degrade load performance due to 
+excessive memory usage. 
+
+  By default CarbonData determines the cardinality at the first data load and allows for dictionary creation only if the cardinality is less than
+1 million.
+
+
+## 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/con/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. |
+| num-executors/executor-cores/executor-memory | spark/con/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. |
+
+
+ 
\ No newline at end of file


[2/9] carbondata-site git commit: added markdowns

Posted by ch...@apache.org.
added markdowns


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

Branch: refs/heads/asf-site
Commit: 7a262a0991efdf2a4d13e8365c236ef7b20504c5
Parents: cf9ebc6
Author: jatin <ja...@knoldus.in>
Authored: Mon May 22 13:55:18 2017 +0530
Committer: jatin <ja...@knoldus.in>
Committed: Mon May 22 13:55:18 2017 +0530

----------------------------------------------------------------------
 content/pdf/maven-pdf-plugin.pdf                | Bin 145272 -> 0 bytes
 src/site/markdown/configuration-parameters.md   | 149 ++++++
 src/site/markdown/data-management.md            | 166 +++++++
 .../markdown/ddl-operation-on-carbondata.md     | 231 ++++++++++
 .../markdown/dml-operation-on-carbondata.md     | 451 +++++++++++++++++++
 src/site/markdown/faq.md                        |  77 ++++
 .../markdown/file-structure-of-carbondata.md    |  45 ++
 src/site/markdown/installation-guide.md         | 200 ++++++++
 src/site/markdown/quick-start-guide.md          | 143 ++++++
 .../supported-data-types-in-carbondata.md       |  41 ++
 src/site/markdown/troubleshooting.md            | 247 ++++++++++
 src/site/markdown/useful-tips-on-carbondata.md  | 180 ++++++++
 12 files changed, 1930 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/7a262a09/content/pdf/maven-pdf-plugin.pdf
----------------------------------------------------------------------
diff --git a/content/pdf/maven-pdf-plugin.pdf b/content/pdf/maven-pdf-plugin.pdf
deleted file mode 100644
index 5f701eb..0000000
Binary files a/content/pdf/maven-pdf-plugin.pdf and /dev/null differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/7a262a09/src/site/markdown/configuration-parameters.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/configuration-parameters.md b/src/site/markdown/configuration-parameters.md
new file mode 100644
index 0000000..774734a
--- /dev/null
+++ b/src/site/markdown/configuration-parameters.md
@@ -0,0 +1,149 @@
+<!--
+    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.
+-->
+
+# Configuring CarbonData
+ This tutorial guides you through the advanced configurations of CarbonData :
+ 
+ * [System Configuration](#system-configuration)
+ * [Performance Configuration](#performance-configuration)
+ * [Miscellaneous Configuration](#miscellaneous-configuration)
+ * [Spark Configuration](#spark-configuration)
+ 
+ 
+##  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 | /user/hive/warehouse/carbon.store | Location where CarbonData will create the store, and write the data in its own format. NOTE: Store location should be in HDFS. |
+| carbon.ddl.base.hdfs.url | hdfs://hacluster/opt/data | 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 | /opt/Carbon/Spark/badrecords | Path where the bad records are stored. |
+| carbon.kettle.home | $SPARK_HOME/carbonlib/carbonplugins | Configuration for loading the data with kettle. |
+| carbon.data.file.version | 2 | 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.|                    
+
+##  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.sort.file.buffer.size | 20 | File read buffer size used during sorting. This value is expressed in MB. | Min=1 and Max=100 |
+| carbon.graph.rowset.size | 100000 | Rowset size exchanged between data load graph steps. | Min=500 and Max=1000000 |
+| carbon.number.of.cores.while.loading | 6 | Number of cores to be used while loading data. |  |
+| carbon.sort.size | 500000 | Record count to sort and write intermediate files to temp. |  |
+| carbon.enableXXHash | true | Algorithm for hashmap for hashkey calculation. |  |
+| carbon.number.of.cores.block.sort | 7 | Number of cores to use for block sort while loading data. |  |
+| 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.update.persist.enable | true | Enabling this parameter considers persistent data. Enabling this will reduce the execution time of UPDATE operation. |  |
+
+
+
+* **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. |
+
+  
+
+* **Query Configuration**
+  
+| Parameter | Default Value | Description | Range |
+|--------------------------------------|---------------|---------------------------------------------------|---------------------------|
+| carbon.number.of.cores | 4 | Number of cores to be used while querying. |  |
+| carbon.inmemory.record.size | 120000 | Number of records to be in memory while querying. | Min=100000 and Max=240000 |
+| carbon.enable.quick.filter | false | Improves the performance of filter query. |  |
+| no.of.cores.to.load.blocks.in.driver | 10 | Number of core to load the blocks in driver. |  |
+
+
+##   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 | 10485760 | File write buffer size used during sorting. |
+| 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.sort.intermediate.files.limit | 20 | Minimum number of intermediate files after which merged sort can be started. |
+| 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. |
+| high.cardinality.value | 100000 | To identify and apply compression for non-high cardinality columns. |
+| carbon.merge.sort.reader.thread | 3 | Maximum no of threads used for reading intermediate files for final merging. |
+| carbon.load.metadata.lock.retries | 3 | Maximum number of retries to get the metadata lock for loading data to table. |
+| carbon.load.metadata.lock.retry.timeout.sec | 5 | Interval between the retries to get the lock. |
+| carbon.tempstore.location | /opt/Carbon/TempStoreLoc | Temporary store location. By default it takes System.getProperty("java.io.tmpdir"). |
+| carbon.load.log.counter | 500000 | Data loading records count logger. |   
+
+
+* **Compaction 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. |
+
+ 
+* **Query Configuration**
+
+| Parameter | Default Value | Description |
+|--------------------------|---------------|-----------------------------------------------------------------------------------------------|
+| 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. | 
+  
+* **Global Dictionary Configurations**
+  
+| Parameter | Default Value | Description |
+|---------------------------------------|---------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
+| high.cardinality.identify.enable | true | If the parameter is true, the high cardinality columns of the dictionary code are automatically recognized and these columns will not be used as global dictionary encoding. If the parameter is false, all dictionary encoding columns are used as dictionary encoding. The high cardinality column must meet the following requirements: value of cardinality > configured value of high.cardinalityEqually, the value of cardinality is higher than the threshold.value of cardinality/ row number x 100 > configured value of high.cardinality.row.count.percentageEqually, the ratio of the cardinality value to data row number is higher than the configured percentage. |
+| high.cardinality.threshold | 1000000 | high.cardinality.threshold | 1000000 | It is a threshold to identify high cardinality of the columns.If the value of columns' cardinality > the configured value, then the columns are excluded from dictionary encoding. |
+| high.cardinality.row.count.percentage | 80 | Percentage to identify whether column cardinality is more than configured percent of total row count.Configuration value formula:Value of cardinality/ row number x 100 > configured value of high.cardinality.row.count.percentageThe value of the parameter must be larger than 0. |
+| carbon.cutOffTimestamp | 1970-01-01 05:30:00 | 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". NOTE: The CarbonData supports data store up to 68 years from the cut-off time defined. For example, if the cut-off time is 1970-01-01 05:30:00, then the data can be stored up to 2038-01-01 05:30:00. |
+| 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>
+ 
+| 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. |
+| spark.sql.bigdata.register.analyseRule | org.apache.spark.sql.hive.acl.CarbonAccessControlRules | CarbonAccessControlRules need to be set for enabling Access Control. |
+   
+ 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/7a262a09/src/site/markdown/data-management.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/data-management.md b/src/site/markdown/data-management.md
new file mode 100644
index 0000000..2663aff
--- /dev/null
+++ b/src/site/markdown/data-management.md
@@ -0,0 +1,166 @@
+<!--
+    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
+This tutorial is going to introduce you to the conceptual details of data management like:
+
+* [Loading Data](#loading-data)
+* [Deleting Data](#deleting-data)
+* [Compacting Data](#compacting-data)
+* [Updating Data](#updating-data)
+
+## Loading Data
+
+* **Scenario**
+
+   After creating a table, you can load data to the table using the [LOAD DATA](dml-operation-on-carbondata.md) 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.
+
+* **Procedure**
+  
+   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.
+   
+   
+| Status | Description |
+|-----------------|------------------------------------------------------------------------------------------------------------|
+| Success | All the data is loaded into table and no bad records found. |
+| Partial Success | Data is loaded into table and bad records are found. Bad records are stored at carbon.badrecords.location. |
+   
+   In case of failure, the error will be logged in error log. Details of loads can be seen with [SHOW SEGMENTS](dml-operation-on-carbondata.md) command. The show segment command output consists of :
+   
+   - SegmentSequenceID
+   - START_TIME OF LOAD
+   - END_TIME OF LOAD 
+   - LOAD STATUS
+ 
+   The latest load will be displayed first in the output.
+   
+   Refer to [DML operations on CarbonData](dml-operation-on-carbondata.md) for load commands.
+   
+   
+## Deleting Data  
+
+* **Scenario**
+   
+   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.
+   
+* **Procedure** 
+
+   The loaded data can be deleted in the following ways:
+   
+   * Delete by Segment ID
+      
+      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.
+      
+| SegmentSequenceId | Status            | Load Start Time      | Load End Time        |
+|-------------------|-------------------|----------------------|----------------------|
+| 0                 | Success           | 2015-11-19 19:14:... | 2015-11-19 19:14:... |
+| 1                 | Marked for Update | 2015-11-19 19:54:... | 2015-11-19 20:08:... |
+| 2                 | Marked for Delete | 2015-11-19 20:25:... | 2015-11-19 20:49:... |
+
+   * Delete by Date Field
+   
+      If the table contains date field, you can delete the data based on a specific date.
+
+   * Delete by Record
+
+      To delete records from CarbonData table based on some filter Condition(s).
+      
+      For delete commands refer to [DML operations on CarbonData](dml-operation-on-carbondata.md).
+      
+   * **NOTE**:
+    
+     - 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.
+     
+     - 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.
+     
+     - If the user wants to force delete the segment physically then he can use CLEAN FILES Command.
+        
+Example :
+          
+```
+CLEAN FILES FOR TABLE table1
+```
+
+ This DML will physically delete the segment which are "Marked for delete" immediately.
+
+## Compacting Data
+      
+* **Scenario**
+  
+  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.  
+      
+* **Procedure**
+
+  There are two types of compaction Minor and Major compaction.
+  
+  - **Minor Compaction**
+    
+     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.
+     
+     - Level 1: Merging of the segments which are not yet compacted.
+     - Level 2: Merging of the compacted segments again to form a bigger segment. 
+  - **Major Compaction**
+     
+     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. 
+      
+   There are number of parameters related to Compaction that can be set in carbon.properties file 
+   
+| Parameter | Default | Application | Description | Valid Values |
+|-----------------------------------------|---------|-------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------|
+| carbon.compaction.level.threshold | 4, 3 | Minor | 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. | NA |
+| carbon.major.compaction.size | 1024 MB | Major | Major compaction size can be configured using this parameter. Sum of the segments which is below this threshold will be merged. | NA |
+| carbon.numberof.preserve.segments | 0 | Minor/Major | 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. | 0-100 |
+| carbon.allowed.compaction.days | 0 | Minor/Major | 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. | 0-100 |
+| carbon.number.of.cores.while.compacting | 2 | Minor/Major | Number of cores which is used to write data during compaction. | 0-100 |   
+  
+   For compaction commands refer to [DDL operations on CarbonData](ddl-operation-on-carbondata.md)
+
+## Updating Data
+
+* **Scenario**
+
+    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.
+
+* **Procedure**
+
+    To update we need to specify the column expression with an optional filter condition(s).
+
+    For update commands refer to [DML operations on CarbonData](dml-operation-on-carbondata.md).
+
+
+    
+
+
+
+
+ 
+ 

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/7a262a09/src/site/markdown/ddl-operation-on-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/ddl-operation-on-carbondata.md b/src/site/markdown/ddl-operation-on-carbondata.md
new file mode 100644
index 0000000..de4999e
--- /dev/null
+++ b/src/site/markdown/ddl-operation-on-carbondata.md
@@ -0,0 +1,231 @@
+<!--
+    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.
+-->
+
+# DDL Operations on CarbonData
+This tutorial guides you through the data definition language support provided by CarbonData.
+
+## Overview 
+The following DDL operations are supported in CarbonData :
+
+* [CREATE TABLE](#create-table)
+* [SHOW TABLE](#show-table)
+* [DROP TABLE](#drop-table)
+* [COMPACTION](#compaction)
+* [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.
+```
+   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
+```
+
+### Parameter Description
+
+| Parameter | Description | Optional |
+|---------------|-----------------------------------------------------------------------------------------------------------------------------------------------|----------|
+| db_name | Name of the database. Database name should consist of alphanumeric characters and underscore(_) special character. | Yes |
+| field_list | Comma separated List of fields with data type. The field names should consist of alphanumeric characters and underscore(_) special character. | No |
+| table_name | The name of the table in Database. Table Name should consist of alphanumeric characters and underscore(_) special character. | No |
+| STORED BY | "org.apache.carbondata.format", identifies and creates a CarbonData table. | No |
+| TBLPROPERTIES | List of CarbonData table properties. |  |
+
+### Usage Guidelines
+
+   Following are the guidelines for using table properties.
+
+   - **Dictionary Encoding Configuration**
+
+       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.
+```
+       TBLPROPERTIES ("DICTIONARY_EXCLUDE"="column1, column2")
+       TBLPROPERTIES ("DICTIONARY_INCLUDE"="column1, column2")
+```
+
+   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.
+
+   - **Row/Column Format Configuration**
+
+       Column groups with more than one column are stored in row format, instead of columnar format. By default, each column is a separate column group.
+```
+TBLPROPERTIES ("COLUMN_GROUPS"="(column1, column3),
+(Column4,Column5,Column6)")
+```
+
+   - **Table Block Size Configuration**
+
+     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.
+```
+       TBLPROPERTIES ("TABLE_BLOCKSIZE"="512 MB")
+```
+
+  Here 512 MB means the block size of this table is 512 MB, you can also set it as 512M or 512.
+
+   - **Inverted Index Configuration**
+
+      Inverted index is very useful to improve compression ratio and query speed, especially for those low-cardinality columns who are in reward position.
+      By default inverted index is enabled. The user can disable the inverted index creation for some columns.
+```
+       TBLPROPERTIES ("NO_INVERTED_INDEX"="column1, column3")
+```
+
+  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.
+
+   NOTE:
+
+   - By default all columns other than numeric datatype are treated as dimensions and all columns of numeric datatype are treated as measures.
+
+   - 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.
+
+### 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 ('COLUMN_GROUPS'='(productNumber,productName)',
+                     'DICTIONARY_EXCLUDE'='storeCity',
+                     'DICTIONARY_INCLUDE'='productNumber',
+                     'NO_INVERTED_INDEX'='productBatch')
+```
+
+## 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];
+```
+
+### Parameter Description
+| Parameter  | Description                                                                               | Optional |
+|------------|-------------------------------------------------------------------------------------------|----------|
+| IN db_Name | Name of the database. Required only if tables of this specific database are to be listed. | Yes      |
+
+### Example:
+```
+  SHOW TABLES IN ProductSchema;
+```
+
+## DROP TABLE
+
+ This command is used to delete an existing table.
+```
+  DROP TABLE [IF EXISTS] [db_name.]table_name;
+```
+
+### Parameter Description
+| Parameter | Description | Optional |
+|-----------|-------------| -------- |
+| db_Name | Name of the database. If not specified, current database will be selected. | YES |
+| table_name | Name of the table to be deleted. | NO |
+
+### Example:
+```
+  DROP TABLE IF EXISTS productSchema.productSalesTable;
+```
+
+## COMPACTION
+
+This command merges the specified number of segments into one segment. This enhances the query performance of the table.
+```
+  ALTER TABLE [db_name.]table_name COMPACT 'MINOR/MAJOR';
+```
+
+  To get details about Compaction refer to [Data Management](data-management.md)
+
+### Parameter Description
+
+| Parameter | Description | Optional |
+| ------------- | -----| ----------- |
+| db_name | Database name, if it is not specified then it uses current database. | YES |
+| table_name | The name of the table in provided database.| NO |
+
+### Syntax
+
+- **Minor Compaction**
+```
+ALTER TABLE table_name COMPACT 'MINOR';
+```
+- **Major Compaction**
+```
+ALTER TABLE table_name COMPACT 'MAJOR';
+```
+
+## 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, a user needs to specify the
+columns to be used for bucketing and the number of buckets. For the selction 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”)
+```
+  
+## Parameter Description
+
+| Parameter 	| Description 	| Optional 	|
+|---------------	|------------------------------------------------------------------------------------------------------------------------------	|----------	|
+| BUCKETNUMBER 	| Specifies the number of Buckets to be created. 	| No 	|
+| BUCKETCOLUMNS 	| Specify the columns to be considered for Bucketing  	| No 	|
+
+## Usage Guidelines
+
+- The feature is supported for Spark 1.6.2 onwards, but the performance optimization is evident from Spark 2.1 onwards.
+
+- Bucketing can not be performed for columns of Complex Data Types.
+
+- Columns in the BUCKETCOLUMN parameter must be only dimension. The BUCKETCOLUMN parameter can not be a measure or a combination of measures and dimensions.
+
+
+## Example :
+
+```
+ CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
+                                productNumber Int,
+                                productName String,
+                                storeCity String,
+                                storeProvince String,
+                                productCategory String,
+                                productBatch String,
+                                saleQuantity Int,
+                                revenue Int)
+   STORED BY 'carbondata'
+   TBLPROPERTIES ('COLUMN_GROUPS'='(productName,productNumber)',
+                  'DICTIONARY_EXCLUDE'='productName',
+                  'DICTIONARY_INCLUDE'='productNumber',
+                  'NO_INVERTED_INDEX'='productBatch',
+                  'BUCKETNUMBER'='4',
+                  'BUCKETCOLUMNS'='productName')
+ ```
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/7a262a09/src/site/markdown/dml-operation-on-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/dml-operation-on-carbondata.md b/src/site/markdown/dml-operation-on-carbondata.md
new file mode 100644
index 0000000..74fa0b0
--- /dev/null
+++ b/src/site/markdown/dml-operation-on-carbondata.md
@@ -0,0 +1,451 @@
+<!--
+    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.
+-->
+
+# DML Operations on CarbonData
+This tutorial guides you through the data manipulation language support provided by CarbonData.
+
+## Overview 
+The following DML operations are supported in CarbonData :
+
+* [LOAD DATA](#load-data)
+* [INSERT DATA INTO A CARBONDATA TABLE](#insert-data-into-a-carbondata-table)
+* [SHOW SEGMENTS](#show-segments)
+* [DELETE SEGMENT BY ID](#delete-segment-by-id)
+* [DELETE SEGMENT BY DATE](#delete-segment-by-date)
+* [UPDATE CARBONDATA TABLE](#update-carbondata-table)
+* [DELETE RECORDS FROM CARBONDATA TABLE](#delete-records-from-carbondata-table)
+
+## LOAD DATA
+
+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 [Data Management](data-management.md) for more details on LOAD.
+
+### Syntax
+
+```
+LOAD DATA [LOCAL] INPATH 'folder_path' 
+INTO TABLE [db_name.]table_name 
+OPTIONS(property_name=property_value, ...)
+```
+
+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.
+
+NOTE: The path shall be canonical path.
+
+### Parameter Description
+
+| Parameter     | Description                                                          | Optional |
+| ------------- | ---------------------------------------------------------------------| -------- |
+| folder_path   | Path of raw csv data folder or file.                                 | NO       |
+| db_name       | Database name, if it is not specified then it uses the current database. | YES      |
+| table_name    | The name of the table in provided database.                          | NO       |
+| OPTIONS       | Extra options provided to Load                                       | YES      |
+ 
+
+### Usage Guidelines
+
+You can use the following options to load data:
+
+- **DELIMITER:** Delimiters can be provided in the load command.
+    
+    ``` 
+    OPTIONS('DELIMITER'=',')
+    ```
+
+- **QUOTECHAR:** Quote Characters can be provided in the load command.
+
+    ```
+    OPTIONS('QUOTECHAR'='"')
+    ```
+
+- **COMMENTCHAR:** Comment Characters can be provided in the load command if user want to comment lines.
+
+    ```
+    OPTIONS('COMMENTCHAR'='#')
+    ```
+
+- **FILEHEADER:** Headers can be provided in the LOAD DATA command if headers are missing in the source files.
+
+    ```
+    OPTIONS('FILEHEADER'='column1,column2') 
+    ```
+
+- **MULTILINE:** CSV with new line character in quotes.
+
+    ```
+    OPTIONS('MULTILINE'='true') 
+    ```
+
+- **ESCAPECHAR:** Escape char can be provided if user want strict validation of escape character on CSV.
+
+    ```
+    OPTIONS('ESCAPECHAR'='\') 
+    ```
+
+- **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:** Date format for specified column.
+
+    ```
+    OPTIONS('DATEFORMAT'='column1:dateFormat1, column2:dateFormat2')
+    ```
+
+    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).
+
+- **USE_KETTLE:** This option is used to specify whether to use kettle for loading data or not. By default kettle is not used for data loading.
+
+    ```
+    OPTIONS('USE_KETTLE'='FALSE')
+    ```
+
+   Note :  It is recommended to set the value for this option as false.
+
+- **SINGLE_PASS:** Single Pass Loading enables single job to finish data loading with dictionary generation on the fly. It enhances performance in the scenarios where the subsequent data loading after initial load involves fewer incremental updates on the dictionary.
+
+   This option specifies whether to use single pass for loading data or not. By default this option is set to FALSE.
+
+    ```
+    OPTIONS('SINGLE_PASS'='TRUE')
+    ```
+
+   Note :
+
+   * If this option is set to TRUE then data loading will take less time.
+
+   * If this option is set to some invalid value other than TRUE or FALSE then it uses the default value.
+### Example:
+
+```
+LOAD DATA local inpath '/opt/rawdata/data.csv' INTO table carbontable
+options('DELIMITER'=',', 'QUOTECHAR'='"','COMMENTCHAR'='#',
+'FILEHEADER'='empno,empname,designation,doj,workgroupcategory,
+ workgroupcategoryname,deptno,deptname,projectcode,
+ projectjoindate,projectenddate,attendance,utilization,salary',
+'MULTILINE'='true','ESCAPECHAR'='\','COMPLEX_DELIMITER_LEVEL_1'='$',
+'COMPLEX_DELIMITER_LEVEL_2'=':',
+'ALL_DICTIONARY_PATH'='/opt/alldictionary/data.dictionary',
+'USE_KETTLE'='FALSE',
+'SINGLE_PASS'='TRUE'
+)
+```
+
+
+## INSERT DATA INTO A 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.
+
+**NOTE** :  The client node where the INSERT command is executing, must be part of the cluster.
+
+### Syntax
+
+```
+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> } ];
+```
+
+### Parameter Description
+
+| Parameter | Description |
+|--------------|---------------------------------------------------------------------------------|
+| CARBON TABLE | The name of the Carbon table in which you want to perform the insert operation. |
+| sourceTableName | The table from which the records are read and inserted into destination CarbonData table. |
+
+### Usage Guidelines
+The following condition must be met for successful insert operation :
+
+- The source table and the CarbonData table must have the same table schema.
+- The table must be created.
+- Overwrite is not supported for CarbonData table.
+- 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.
+- 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.
+
+To enable data load or update during insert operation, configure the following property to true.
+
+```
+carbon.insert.persist.enable=true
+```
+
+By default the above configuration will be false.
+
+**NOTE**: Enabling this property will reduce the performance.
+
+### 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 INTO table1 SELECT * FROM table2 
+where exists (select * from table3 
+where table2.item1 = table3.item1);
+```
+
+**The Status Success/Failure shall be captured in the driver log.**
+
+## SHOW SEGMENTS
+
+This command is used to get the segments of CarbonData table.
+
+```
+SHOW SEGMENTS FOR TABLE [db_name.]table_name 
+LIMIT number_of_segments;
+```
+
+### Parameter Description
+
+| Parameter          | Description                                                          | Optional |
+| ------------------ | ---------------------------------------------------------------------| ---------|
+| db_name            | Database name, if it is not specified then it uses the current database. | YES      |
+| table_name         | The name of the table in provided database.                          | NO       |
+| number_of_segments | Limit the output to this number.                                     | YES      |
+
+### Example:
+
+```
+SHOW SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4;
+```
+
+## DELETE SEGMENT BY ID
+
+This command is used to delete segment by using the segment ID. Each segment has a unique segment ID associated with it. 
+Using this segment ID, you can remove the segment.
+
+The following command will get the segmentID.
+
+```
+SHOW SEGMENTS FOR Table dbname.tablename 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 SEGMENT segment_sequence_id1, segments_sequence_id2, .... 
+FROM TABLE tableName
+```
+
+### Parameter Description
+| Parameter  | Description                                                          | Optional |
+| -----------| ---------------------------------------------------------------------|----------|
+| segment_id | Segment Id of the load.                                              | NO       |
+| db_name    | Database name, if it is not specified then it uses the current database. | YES      |
+| table_name | The name of the table in provided database.                          | NO       |
+
+### Example:
+
+```
+DELETE SEGMENT 0 FROM TABLE CarbonDatabase.CarbonTable;
+DELETE SEGMENT 0.1,5,8 FROM TABLE CarbonDatabase.CarbonTable;
+```
+  NOTE: Here 0.1 is compacted segment sequence id. 
+
+## 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 [schema_name.]table_name 
+WHERE[DATE_FIELD]BEFORE [DATE_VALUE]
+```
+
+### Parameter Description
+
+| Parameter  | Description                                                                                        | Optional |
+| ---------- | ---------------------------------------------------------------------------------------------------| -------- |
+| DATE_VALUE | Valid segment load start time value. All the segments before this specified date will be deleted. | NO       |
+| db_name    | Database name, if it is not specified then it uses the current database.                               | YES      |
+| table_name | The name of the table in provided database.                                                        | NO       |
+
+### Example:
+
+```
+ DELETE SEGMENTS FROM TABLE CarbonDatabase.CarbonTable 
+ WHERE STARTTIME BEFORE '2017-06-01 12:05:06';  
+```
+
+## Update CarbonData Table
+This command will allow to update the carbon table based on the column expression and optional filter conditions.
+
+### Syntax
+
+```
+ UPDATE <table_name>
+ SET (column_name1, column_name2, ... column_name n) =
+ (column1_expression , column2_expression . .. column n_expression )
+ [ WHERE { <filter_condition> } ];
+```
+
+alternatively the following the command can also be used for updating the CarbonData Table :
+
+```
+UPDATE <table_name>
+SET (column_name1, column_name2,) =
+(select sourceColumn1, sourceColumn2 from sourceTable
+[ WHERE { <filter_condition> } ] )
+[ WHERE { <filter_condition> } ];
+```
+
+### Parameter Description
+
+| Parameter | Description |
+|--------------|---------------------------------------------------------------------------------|
+| table_name | The name of the Carbon table in which you want to perform the update operation. |
+| column_name | The destination columns to be updated. |
+| sourceColumn | The source table column values to be updated in destination table. |
+| sourceTable | The table from which the records are updated into destination Carbon table. |
+
+### Usage Guidelines
+The following conditions must be met for successful updation :
+
+- The update command fails if multiple input rows in source table are matched with single row in destination table.
+- If the source table generates empty records, the update operation will complete successfully without updating the table.
+- 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.
+- In sub-query, if the source table and the target table are same, then the update operation fails.
+- If the sub-query used in UPDATE statement contains aggregate method or group by query, then the UPDATE operation fails.
+
+### Examples
+
+ Update is not supported for queries that contain aggregate or group by.
+
+```
+ 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);
+```
+
+Here the Update Operation fails as the query contains aggregate function sum(b.profit) and group by clause in the sub-query.
+
+
+```
+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 > 1);
+```
+
+
+```
+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 > 1);
+```
+
+
+```
+UPDATE carbonTable1 SET (c2, c5 ) = (c2 + 1, concat(c5 , "y" ));
+```
+
+
+```
+UPDATE carbonTable1 d SET (c2, c5 ) = (c2 + 1, "xyx")
+WHERE d.column1 = 'india';
+```
+
+
+```
+UPDATE carbonTable1 d SET (c2, c5 ) = (c2 + 1, "xyx")
+WHERE d.column1 = 'india'
+and EXISTS( SELECT * FROM table3 o WHERE o.column2 > 1);
+```
+
+**The Status Success/Failure shall be captured in the driver log and the client.**
+
+
+## Delete Records from CarbonData Table
+This command allows us to delete records from CarbonData table.
+
+### Syntax
+
+```
+DELETE FROM table_name [WHERE expression];
+```
+
+### Parameter Description
+
+| Parameter | Description |
+|--------------|-----------------------------------------------------------------------|
+| table_name | The name of the Carbon table in which you want to perform the delete. |
+
+
+### Examples
+
+```
+DELETE FROM columncarbonTable1 d WHERE d.column1  = 'china';
+```
+
+```
+DELETE FROM dest WHERE column1 IN ('china', 'USA');
+```
+
+```
+DELETE FROM columncarbonTable1
+WHERE column1 IN (SELECT column11 FROM sourceTable2);
+```
+
+```
+DELETE FROM columncarbonTable1
+WHERE column1 IN (SELECT column11 FROM sourceTable2 WHERE
+column1 = 'USA');
+```
+
+```
+DELETE FROM columncarbonTable1 WHERE column2 >= 4
+```
+
+**The Status Success/Failure shall be captured in the driver log and the client.**

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/7a262a09/src/site/markdown/faq.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/faq.md b/src/site/markdown/faq.md
new file mode 100644
index 0000000..57ac171
--- /dev/null
+++ b/src/site/markdown/faq.md
@@ -0,0 +1,77 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+-->
+
+# FAQs
+
+* [What are Bad Records?](#what-are-bad-records)
+* [Where are Bad Records Stored in CarbonData?](#where-are-bad-records-stored-in-carbondata)
+* [How to enable Bad Record Logging?](#how-to-enable-bad-record-logging)
+* [How to ignore the Bad Records?](#how-to-ignore-the-bad-records)
+* [How to specify store location while creating carbon session?](#how-to-specify-store-location-while-creating-carbon-session)
+* [What is Carbon Lock Type?](#what-is-carbon-lock-type)
+* [How to resolve Abstract Method Error?](#how-to-resolve-abstract-method-error)
+
+## What are Bad Records?
+Records that fail to get loaded into the CarbonData due to data type incompatibility or are empty or have incompatible format are classified as Bad Records.
+
+## Where are Bad Records Stored in CarbonData?
+The bad records are stored at the location set in carbon.badRecords.location in carbon.properties file.
+By default **carbon.badRecords.location** specifies the following location ``/opt/Carbon/Spark/badrecords``.
+
+## How to enable Bad Record Logging?
+While loading data we can specify the approach to handle Bad Records. In order to analyse the cause of the Bad Records the parameter ``BAD_RECORDS_LOGGER_ENABLE`` must be set to value ``TRUE``. There are multiple approaches to handle Bad Records which can be specified  by the parameter ``BAD_RECORDS_ACTION``.
+
+- To pad the incorrect values of the csv rows with NULL value and load the data in CarbonData, set the following in the query :
+```
+'BAD_RECORDS_ACTION'='FORCE'
+```
+
+- To write the Bad Records without padding incorrect values with NULL in the raw csv (set in the parameter **carbon.badRecords.location**), set the following in the query :
+```
+'BAD_RECORDS_ACTION'='REDIRECT'
+```
+
+## How to ignore the Bad Records?
+To ignore the Bad Records from getting stored in the raw csv, we need to set the following in the query :
+```
+'BAD_RECORDS_ACTION'='IGNORE'
+```
+
+## How to specify store location while creating carbon session?
+The store location specified while creating carbon session is used by the CarbonData to store the meta data like the schema, dictionary files, dictionary meta data and sort indexes.
+
+Try creating ``carbonsession`` with ``storepath`` specified in the following manner :
+```
+val carbon = SparkSession.builder().config(sc.getConf).getOrCreateCarbonSession(<store_path>)
+```
+Example:
+```
+val carbon = SparkSession.builder().config(sc.getConf).getOrCreateCarbonSession("hdfs://localhost:9000/carbon/store ")
+```
+
+## What is Carbon Lock Type?
+The Apache CarbonData acquires lock on the files to prevent concurrent operation from modifying the same files. The lock can be of the following types depending on the storage location, for HDFS we specify it to be of type HDFSLOCK. By default it is set to type LOCALLOCK.
+The property carbon.lock.type configuration specifies the type of lock to be acquired during concurrent operations on table. This property can be set with the following values :
+- **LOCALLOCK** : This Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently.
+- **HDFSLOCK** : This Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData spark applications are launched and no ZooKeeper is running on cluster and the HDFS supports, file based locking.
+
+## How to resolve Abstract Method Error?
+In order to build CarbonData project it is necessary to specify the spark profile. The spark profile sets the Spark Version. You need to specify the ``spark version`` while using Maven to build project.
+
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/7a262a09/src/site/markdown/file-structure-of-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/file-structure-of-carbondata.md b/src/site/markdown/file-structure-of-carbondata.md
new file mode 100644
index 0000000..bfbcee4
--- /dev/null
+++ b/src/site/markdown/file-structure-of-carbondata.md
@@ -0,0 +1,45 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+-->
+
+# CarbonData File Structure
+
+CarbonData files contain groups of data called blocklets, along with all required information like schema, offsets and indices etc, in a file header and footer, co-located in HDFS.
+
+The file footer can be read once to build the indices in memory, which can be utilized for optimizing the scans and processing for all subsequent queries.
+
+### Understanding CarbonData File Structure
+* Block : It would be as same as HDFS block, CarbonData creates one file for each data block, user can specify TABLE_BLOCKSIZE during creation table. Each file contains File Header, Blocklets and File Footer. 
+
+![CarbonData File Structure](../docs/images/carbon_data_file_structure_new.png?raw=true)
+
+* 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 to pages, it is guaranteed to be contiguous in file.
+* Page : It has the data of one column and the number of row is fixed to 32000 size. 
+
+![CarbonData File Format](../docs/images/carbon_data_format_new.png?raw=true)
+
+### Each page contains three types of data
+* Data Page: Contains the encoded data of a column of columns.
+* Row ID Page (optional): Contains the row ID mappings used when the data page is stored as an inverted index.
+* RLE Page (optional): Contains additional metadata used when the data page is RLE coded.
+
+
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/7a262a09/src/site/markdown/installation-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/installation-guide.md b/src/site/markdown/installation-guide.md
new file mode 100644
index 0000000..c5bf6df
--- /dev/null
+++ b/src/site/markdown/installation-guide.md
@@ -0,0 +1,200 @@
+<!--
+    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/incubator-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. Copy the `./processing/carbonplugins` folder from CarbonData repository to `$SPARK_HOME/carbonlib/` folder.
+
+    **NOTE**: carbonplugins will contain .kettle folder.
+
+6. Repeat Step 2 to Step 5 in all the nodes of the cluster.
+    
+7. In Spark node[master], configure the properties mentioned in the following table in `$SPARK_HOME/conf/spark-defaults.conf` file.
+
+   | Property | Value | Description |
+   |---------------------------------|-----------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------|
+   | carbon.kettle.home | `$SPARK_HOME/carbonlib/carbonplugins` | Path that will be used by CarbonData internally to create graph for loading the data |
+   | 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. |
+
+8. Add the following properties in `$SPARK_HOME/conf/carbon.properties` file:
+
+   | Property             | Required | Description                                                                            | Example                             | Remark  |
+   |----------------------|----------|----------------------------------------------------------------------------------------|-------------------------------------|---------|
+   | carbon.storelocation | NO       | Location where data CarbonData will create the store and write the data in its own format. | hdfs://HOSTNAME:PORT/Opt/CarbonStore      | Propose to set HDFS directory |
+   | carbon.kettle.home   | YES      | Path that will be used by CarbonData internally to create graph for loading the data.         | `$SPARK_HOME/carbonlib/carbonplugins` |         |
+
+
+9. Verify the installation. For example:
+
+   ```
+   ./spark-shell --master spark://HOSTNAME:PORT --total-executor-cores 2
+   --executor-memory 2G
+   ```
+
+**NOTE**: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.
+
+To get started with CarbonData : [Quick Start](quick-start-guide.md), [DDL Operations on CarbonData](ddl-operation-on-carbondata.md)
+
+## Installing and Configuring CarbonData on "Spark on YARN" Cluster
+
+   This section provides the procedure to install CarbonData on "Spark on YARN" cluster.
+
+### Prerequisites
+   * Hadoop HDFS and Yarn should be installed and running.
+   * Spark should be installed and running in all the clients.
+   * CarbonData user should have permission to access HDFS.
+
+### Procedure
+
+   The following steps are only for Driver Nodes. (Driver nodes are the one which starts the spark context.)
+
+1. [Build the CarbonData](https://github.com/apache/incubator-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 `./processing/carbonplugins` folder from CarbonData repository to `$SPARK_HOME/carbonlib/` folder.
+
+    **NOTE**: carbonplugins will contain .kettle folder.
+
+3. Copy the `./conf/carbon.properties.template` file from CarbonData repository to `$SPARK_HOME/conf/` folder and rename the file to `carbon.properties`.
+
+4. 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/
+    ```
+
+5. 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/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` |
+
+
+6. Add the following properties in `$SPARK_HOME/conf/carbon.properties`:
+
+   | Property | Required | Description | Example | Default Value |
+   |----------------------|----------|----------------------------------------------------------------------------------------|-------------------------------------|---------------|
+   | carbon.storelocation | NO | Location where CarbonData will create the store and write the data in its own format. | hdfs://HOSTNAME:PORT/Opt/CarbonStore | Propose to set HDFS directory|
+   | carbon.kettle.home | YES | Path that will be used by CarbonData internally to create graph for loading the data. | carbondata.tar.gz/carbonlib/carbonplugins |  |
+
+
+7. Verify the installation.
+
+   ```
+     ./bin/spark-shell --master yarn-client --driver-memory 1g
+     --executor-cores 2 --executor-memory 2G
+   ```
+  **NOTE**: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.
+
+  Getting started with CarbonData : [Quick Start](quick-start-guide.md), [DDL Operations on CarbonData](ddl-operation-on-carbondata.md)
+
+## Query Execution Using CarbonData Thrift Server
+
+### Starting CarbonData Thrift Server.
+
+   a. cd `$SPARK_HOME`
+
+   b. Run the following command to start the CarbonData thrift server.
+
+   ```
+   ./bin/spark-submit --conf spark.sql.hive.thriftServer.singleSession=true
+   --class org.apache.carbondata.spark.thriftserver.CarbonThriftServer
+   $SPARK_HOME/carbonlib/$CARBON_ASSEMBLY_JAR <carbon_store_path>
+   ```
+
+| Parameter | Description | Example |
+|---------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------|
+| CARBON_ASSEMBLY_JAR | CarbonData assembly jar name present in the `$SPARK_HOME/carbonlib/` folder. | carbondata_2.10-0.1.0-incubating-SNAPSHOT-shade-hadoop2.7.2.jar |
+| carbon_store_path | This is a parameter to the CarbonThriftServer class. This a HDFS path where CarbonData files will be kept. Strongly Recommended to put same as carbon.storelocation parameter of carbon.properties. | `hdfs://<host_name>:port/user/hive/warehouse/carbon.store` |
+
+**Examples**
+   
+   * Start with default memory and executors.
+
+```
+./bin/spark-submit --conf spark.sql.hive.thriftServer.singleSession=true 
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer 
+$SPARK_HOME/carbonlib
+/carbondata_2.10-0.1.0-incubating-SNAPSHOT-shade-hadoop2.7.2.jar 
+hdfs://<host_name>:port/user/hive/warehouse/carbon.store
+```
+   
+   * Start with Fixed executors and resources.
+
+```
+./bin/spark-submit --conf spark.sql.hive.thriftServer.singleSession=true 
+--class org.apache.carbondata.spark.thriftserver.CarbonThriftServer 
+--num-executors 3 --driver-memory 20g --executor-memory 250g 
+--executor-cores 32 
+/srv/OSCON/BigData/HACluster/install/spark/sparkJdbc/lib
+/carbondata_2.10-0.1.0-incubating-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
+     ./bin/beeline jdbc:hive2://<thrftserver_host>:port
+
+     Example
+     ./bin/beeline jdbc:hive2://10.10.10.10:10000
+```
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/7a262a09/src/site/markdown/quick-start-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/quick-start-guide.md b/src/site/markdown/quick-start-guide.md
new file mode 100644
index 0000000..c29a8d3
--- /dev/null
+++ b/src/site/markdown/quick-start-guide.md
@@ -0,0 +1,143 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+-->
+
+# Quick Start
+This tutorial provides a quick introduction to using CarbonData.
+
+##  Prerequisites
+* [Installation and building CarbonData](https://github.com/apache/incubator-carbondata/blob/master/build).
+* Create a sample.csv file using the following commands. The CSV file is required for loading data into CarbonData.
+
+  ```
+  cd carbondata
+  cat > sample.csv << EOF
+  id,name,city,age
+  1,david,shenzhen,31
+  2,eason,shenzhen,27
+  3,jarry,wuhan,35
+  EOF
+  ```
+
+## Interactive Analysis with Spark Shell
+
+## Version 2.1
+
+Apache Spark Shell provides a simple way to learn the API, as well as a powerful tool to analyze data interactively. Please visit [Apache Spark Documentation](http://spark.apache.org/docs/latest/) for more details on Spark shell.
+
+#### Basics
+
+Start Spark shell by running the following command in the Spark directory:
+
+```
+./bin/spark-shell --jars <carbondata assembly jar path>
+```
+
+In this shell, SparkSession is readily available as `spark` and Spark context is readily available as `sc`.
+
+In order to create a CarbonSession we will have to configure it explicitly in the following manner :
+
+* Import the following :
+
+```
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.CarbonSession._
+```
+
+* Create a CarbonSession :
+
+```
+val carbon = SparkSession.builder().config(sc.getConf).getOrCreateCarbonSession("<hdfs store path>")
+```
+**NOTE**: By default metastore location is pointed to `../carbon.metastore`, user can provide own metastore location to CarbonSession like `SparkSession.builder().config(sc.getConf).getOrCreateCarbonSession("<hdfs store path>", "<local metastore path>")`
+
+#### Executing Queries
+
+##### Creating a Table
+
+```
+scala>carbon.sql("CREATE TABLE IF NOT EXISTS test_table(id string, name string, city string, age Int) STORED BY 'carbondata'")
+```
+
+##### Loading Data to a Table
+
+```
+scala>carbon.sql("LOAD DATA INPATH 'sample.csv file path' INTO TABLE test_table")
+```
+**NOTE**: Please provide the real file path of `sample.csv` for the above script.
+
+###### Query Data from a Table
+
+```
+scala>carbon.sql("SELECT * FROM test_table").show()
+
+scala>carbon.sql("SELECT city, avg(age), sum(age) FROM test_table GROUP BY city").show()
+```
+
+## Interactive Analysis with Spark Shell
+## Version 1.6
+
+#### Basics
+
+Start Spark shell by running the following command in the Spark directory:
+
+```
+./bin/spark-shell --jars <carbondata assembly jar path>
+```
+
+**NOTE**: In this shell, SparkContext is readily available as `sc`.
+
+* In order to execute the Queries we need to import CarbonContext:
+
+```
+import org.apache.spark.sql.CarbonContext
+```
+
+* Create an instance of CarbonContext in the following manner :
+
+```
+val cc = new CarbonContext(sc, "<hdfs store path>")
+```
+**NOTE**: If running on local machine without hdfs, configure the local machine's store path instead of hdfs store path
+
+#### Executing Queries
+
+##### Creating a Table
+
+```
+scala>cc.sql("CREATE TABLE IF NOT EXISTS test_table (id string, name string, city string, age Int) STORED BY 'carbondata'")
+```
+To see the table created :
+
+```
+scala>cc.sql("SHOW TABLES").show()
+```
+
+##### Loading Data to a Table
+
+```
+scala>cc.sql("LOAD DATA INPATH 'sample.csv file path' INTO TABLE test_table")
+```
+**NOTE**: Please provide the real file path of `sample.csv` for the above script.
+
+##### Query Data from a Table
+
+```
+scala>cc.sql("SELECT * FROM test_table").show()
+scala>cc.sql("SELECT city, avg(age), sum(age) FROM test_table GROUP BY city").show()
+```

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/7a262a09/src/site/markdown/supported-data-types-in-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/supported-data-types-in-carbondata.md b/src/site/markdown/supported-data-types-in-carbondata.md
new file mode 100644
index 0000000..8f271e3
--- /dev/null
+++ b/src/site/markdown/supported-data-types-in-carbondata.md
@@ -0,0 +1,41 @@
+<!--
+    Licensed to the Apache Software Foundation (ASF) under one
+    or more contributor license agreements.  See the NOTICE file
+    distributed with this work for additional information
+    regarding copyright ownership.  The ASF licenses this file
+    to you under the Apache License, Version 2.0 (the
+    "License"); you may not use this file except in compliance
+    with the License.  You may obtain a copy of the License at
+
+      http://www.apache.org/licenses/LICENSE-2.0
+
+    Unless required by applicable law or agreed to in writing,
+    software distributed under the License is distributed on an
+    "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+    KIND, either express or implied.  See the License for the
+    specific language governing permissions and limitations
+    under the License.
+-->
+
+#  Data Types
+
+#### CarbonData supports the following data types:
+
+  * Numeric Types
+    * SMALLINT
+    * INT/INTEGER
+    * BIGINT
+    * DOUBLE
+    * DECIMAL
+
+  * Date/Time Types
+    * TIMESTAMP
+    * DATE
+
+  * String Types
+    * STRING
+    * CHAR
+
+  * Complex Types
+    * arrays: ARRAY``<data_type>``
+    * structs: STRUCT``<col_name : data_type COMMENT col_comment, ...>``

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/7a262a09/src/site/markdown/troubleshooting.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/troubleshooting.md b/src/site/markdown/troubleshooting.md
new file mode 100644
index 0000000..9181d83
--- /dev/null
+++ b/src/site/markdown/troubleshooting.md
@@ -0,0 +1,247 @@
+<!--
+    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.
+
+## 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.
+
+## Failure to read the metastore db during table creation.
+
+  **Symptom**
+
+  We get the following exception on trying to connect :
+
+  ```
+  Cannot read the metastore db
+  ```
+
+  **Possible Cause**
+
+  The metastore db is dysfunctional.
+
+  **Procedure**
+
+  Remove the metastore db from the carbon.metastore in the Spark Directory.
+
+## Failed to load data on the cluster
+
+  **Symptom**
+
+  Data loading fails with the following exception :
+
+   ```
+   Data Load failure exeception
+   ```
+
+  **Possible Cause**
+
+  The following issue can cause the failure :
+
+  1. The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.
+
+  2. Path to hdfs ddl is not configured correctly in the carbon.properties.
+
+  **Procedure**
+
+   Follow the steps to ensure the following configuration files are consistent across all the nodes:
+
+   1. Copy the core-site.xml, hive-site.xml, yarn-site,carbon.properties files from the master node to all the other nodes in the cluster.
+      For example, you can use scp to copy this file to all the nodes.
+
+      Note : Set the path to hdfs ddl in carbon.properties in the master node.
+
+   2. For the changes to take effect, restart the Spark cluster.
+
+
+
+## Failed to insert data on the cluster
+
+  **Symptom**
+
+  Insertion fails with the following exception :
+
+   ```
+   Data Load failure exeception
+   ```
+
+  **Possible Cause**
+
+  The following issue can cause the failure :
+
+  1. The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.
+
+  2. Path to hdfs ddl is not configured correctly in the carbon.properties.
+
+  **Procedure**
+
+   Follow the steps to ensure the following configuration files are consistent across all the nodes:
+
+   1. Copy the core-site.xml, hive-site.xml, yarn-site,carbon.properties files from the master node to all the other nodes in the cluster.
+      For example, you can use scp to copy this file to all the nodes.
+
+      Note : Set the path to hdfs ddl in carbon.properties in the master node.
+
+   2. For the changes to take effect, restart the Spark cluster.
+
+## Failed to execute Concurrent Operations(Load,Insert,Update) on table by multiple workers.
+
+  **Symptom**
+
+  Execution fails with the following exception :
+
+   ```
+   Table is locked for updation.
+   ```
+
+  **Possible Cause**
+
+  Concurrency not supported.
+
+  **Procedure**
+
+  Worker must wait for the query execution to complete and the table to release the lock for another query execution to succeed..
+
+## Failed to create a table with a single numeric column.
+
+  **Symptom**
+
+  Execution fails with the following exception :
+
+   ```
+   Table creation fails.
+   ```
+
+  **Possible Cause**
+
+  Behavior not supported.
+
+  **Procedure**
+
+  A single column that can be considered as dimension is mandatory for table creation.


[9/9] carbondata-site git commit: Added functionality to auto generate PDF for Documentation

Posted by ch...@apache.org.
Added functionality to auto generate PDF for Documentation


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

Branch: refs/heads/asf-site
Commit: 20e101ec64a79e92b08cd402bf16fd4af5be8c29
Parents: 3d4c33e
Author: chenliang613 <ch...@apache.org>
Authored: Wed May 24 12:21:21 2017 +0800
Committer: chenliang613 <ch...@apache.org>
Committed: Wed May 24 12:21:21 2017 +0800

----------------------------------------------------------------------
 content/WEB-INF/classes/CleanUp$.class          | Bin 0 -> 3370 bytes
 content/WEB-INF/classes/CleanUp.class           | Bin 0 -> 719 bytes
 content/WEB-INF/classes/MDFileConverter.class   | Bin 0 -> 8088 bytes
 content/WEB-INF/classes/META-INF/DEPENDENCIES   | 113 +++++++++++
 content/WEB-INF/classes/META-INF/LICENSE        | 202 +++++++++++++++++++
 content/WEB-INF/classes/META-INF/NOTICE         |   8 +
 content/WEB-INF/classes/MdFileHandler.class     | Bin 0 -> 6144 bytes
 content/WEB-INF/classes/Module.class            | Bin 0 -> 1112 bytes
 content/WEB-INF/classes/WebsiteLauncher$.class  | Bin 0 -> 2104 bytes
 content/WEB-INF/classes/WebsiteLauncher.class   | Bin 0 -> 713 bytes
 content/WEB-INF/classes/application.conf        |  20 ++
 .../WEB-INF/classes/services/ConfService.class  | Bin 0 -> 825 bytes
 .../classes/services/ConfServiceImpl.class      | Bin 0 -> 2905 bytes
 .../WEB-INF/classes/services/DataService.class  | Bin 0 -> 781 bytes
 .../classes/services/DataServiceImpl.class      | Bin 0 -> 4358 bytes
 .../WEB-INF/classes/services/FileService.class  | Bin 0 -> 708 bytes
 .../classes/services/FileServiceImpl.class      | Bin 0 -> 3052 bytes
 content/WEB-INF/lib/aopalliance-1.0.jar         | Bin 0 -> 4467 bytes
 content/WEB-INF/lib/classworlds-1.1-alpha-2.jar | Bin 0 -> 37516 bytes
 content/WEB-INF/lib/commons-codec-1.9.jar       | Bin 0 -> 263965 bytes
 content/WEB-INF/lib/commons-logging-1.2.jar     | Bin 0 -> 61829 bytes
 content/WEB-INF/lib/config-1.3.1.jar            | Bin 0 -> 282549 bytes
 content/WEB-INF/lib/guice-3.0.jar               | Bin 0 -> 710492 bytes
 content/WEB-INF/lib/httpclient-4.5.3.jar        | Bin 0 -> 747794 bytes
 content/WEB-INF/lib/httpcore-4.4.6.jar          | Bin 0 -> 323824 bytes
 content/WEB-INF/lib/javax.inject-1.jar          | Bin 0 -> 2497 bytes
 content/WEB-INF/lib/junit-3.8.1.jar             | Bin 0 -> 121070 bytes
 content/WEB-INF/lib/maven-artifact-3.0.jar      | Bin 0 -> 51908 bytes
 content/WEB-INF/lib/maven-model-3.0.jar         | Bin 0 -> 164685 bytes
 content/WEB-INF/lib/maven-plugin-api-3.0.jar    | Bin 0 -> 48920 bytes
 content/WEB-INF/lib/mockito-all-1.9.5.jar       | Bin 0 -> 1581066 bytes
 .../WEB-INF/lib/plexus-classworlds-2.2.3.jar    | Bin 0 -> 46099 bytes
 .../lib/plexus-component-annotations-1.5.4.jar  | Bin 0 -> 4208 bytes
 ...s-container-default-1.0-alpha-9-stable-1.jar | Bin 0 -> 194185 bytes
 .../lib/plexus-resources-1.0-alpha-7.jar        | Bin 0 -> 23259 bytes
 content/WEB-INF/lib/plexus-utils-1.5.5.jar      | Bin 0 -> 251275 bytes
 content/WEB-INF/lib/scala-library-2.12.1.jar    | Bin 0 -> 5272325 bytes
 .../lib/scala-parser-combinators_2.11-1.0.1.jar | Bin 0 -> 419701 bytes
 .../lib/scala-parser-combinators_2.12-1.0.4.jar | Bin 0 -> 204313 bytes
 content/WEB-INF/lib/scala-reflect-2.12.0.jar    | Bin 0 -> 3574999 bytes
 content/WEB-INF/lib/scala-xml_2.11-1.0.1.jar    | Bin 0 -> 655685 bytes
 content/WEB-INF/lib/scala-xml_2.12-1.0.5.jar    | Bin 0 -> 548430 bytes
 content/WEB-INF/lib/scalactic_2.12-3.0.1.jar    | Bin 0 -> 705564 bytes
 content/WEB-INF/lib/scalariform_2.11-0.1.7.jar  | Bin 0 -> 1934192 bytes
 .../lib/scalastyle-maven-plugin-0.8.0.jar       | Bin 0 -> 18780 bytes
 content/WEB-INF/lib/scalastyle_2.11-0.8.0.jar   | Bin 0 -> 894844 bytes
 content/WEB-INF/lib/scalatest_2.12-3.0.1.jar    | Bin 0 -> 6992597 bytes
 content/WEB-INF/lib/sisu-guice-2.1.7-noaop.jar  | Bin 0 -> 471737 bytes
 content/WEB-INF/lib/sisu-inject-bean-1.4.2.jar  | Bin 0 -> 152989 bytes
 .../WEB-INF/lib/sisu-inject-plexus-1.4.2.jar    | Bin 0 -> 201716 bytes
 content/WEB-INF/lib/slf4j-api-1.7.25.jar        | Bin 0 -> 41203 bytes
 content/WEB-INF/lib/slf4j-simple-1.7.25.jar     | Bin 0 -> 15257 bytes
 content/pdf/maven-pdf-plugin.pdf                | Bin 0 -> 193057 bytes
 53 files changed, 343 insertions(+)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/CleanUp$.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/CleanUp$.class b/content/WEB-INF/classes/CleanUp$.class
new file mode 100644
index 0000000..099f826
Binary files /dev/null and b/content/WEB-INF/classes/CleanUp$.class differ

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

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

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/META-INF/DEPENDENCIES
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/META-INF/DEPENDENCIES b/content/WEB-INF/classes/META-INF/DEPENDENCIES
new file mode 100644
index 0000000..dd8223b
--- /dev/null
+++ b/content/WEB-INF/classes/META-INF/DEPENDENCIES
@@ -0,0 +1,113 @@
+// ------------------------------------------------------------------
+// Transitive dependencies of this project determined from the
+// maven pom organized by organization.
+// ------------------------------------------------------------------
+
+Apache CarbonData :: Website
+
+
+From: 'an unknown organization'
+  - AOP alliance (http://aopalliance.sourceforge.net) aopalliance:aopalliance:jar:1.0
+    License: Public Domain 
+  - javax.inject (http://code.google.com/p/atinject/) javax.inject:javax.inject:jar:1
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+  - Mockito (http://www.mockito.org) org.mockito:mockito-all:jar:1.9.5
+    License: The MIT License  (http://code.google.com/p/mockito/wiki/License)
+  - Sisu - Guice (http://forge.sonatype.com/sisu-guice/) org.sonatype.sisu:sisu-guice:pom:2.1.7
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+
+From: 'Codehaus' (http://www.codehaus.org/)
+  - Plexus Classworlds (http://plexus.codehaus.org/plexus-classworlds/) org.codehaus.plexus:plexus-classworlds:jar:2.2.3
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+  - Plexus :: Component Annotations (http://plexus.codehaus.org/plexus-containers/plexus-component-annotations/) org.codehaus.plexus:plexus-component-annotations:jar:1.5.4
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+  - Default Plexus Container  org.codehaus.plexus:plexus-container-default:jar:1.0-alpha-9-stable-1
+
+  - Plexus Resource Component (http://plexus.codehaus.org/plexus-components/plexus-resources) org.codehaus.plexus:plexus-resources:jar:1.0-alpha-7
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+  - Plexus Common Utilities (http://plexus.codehaus.org/plexus-utils) org.codehaus.plexus:plexus-utils:jar:1.5.5
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+
+From: 'com.typesafe'
+  - config (https://github.com/typesafehub/config) com.typesafe:config:bundle:1.3.1
+    License: Apache License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0)
+
+From: 'Google, Inc.' (http://www.google.com)
+  - Google Guice - Core Library (http://code.google.com/p/google-guice/guice/) com.google.inject:guice:jar:3.0
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+
+From: 'JUnit' (http://www.junit.org)
+  - JUnit (http://junit.org) junit:junit:jar:3.8.1
+    License: Common Public License Version 1.0  (http://www.opensource.org/licenses/cpl1.0.txt)
+
+From: 'LAMP/EPFL' (http://lamp.epfl.ch/)
+  - Scala Library (http://www.scala-lang.org/) org.scala-lang:scala-library:jar:2.12.1
+    License: BSD 3-Clause  (http://www.scala-lang.org/license.html)
+  - Scala Compiler (http://www.scala-lang.org/) org.scala-lang:scala-reflect:jar:2.12.0
+    License: BSD 3-Clause  (http://www.scala-lang.org/license.html)
+
+From: 'org.scala-lang.modules' (http://www.scala-lang.org/)
+  - scala-parser-combinators (http://www.scala-lang.org/) org.scala-lang.modules:scala-parser-combinators_2.11:bundle:1.0.1
+    License: BSD 3-clause  (http://opensource.org/licenses/BSD-3-Clause)
+  - scala-parser-combinators (http://www.scala-lang.org/) org.scala-lang.modules:scala-parser-combinators_2.12:bundle:1.0.4
+    License: BSD 3-clause  (http://opensource.org/licenses/BSD-3-Clause)
+  - scala-xml (http://www.scala-lang.org/) org.scala-lang.modules:scala-xml_2.11:bundle:1.0.1
+    License: BSD 3-clause  (http://opensource.org/licenses/BSD-3-Clause)
+  - scala-xml (http://www.scala-lang.org/) org.scala-lang.modules:scala-xml_2.12:bundle:1.0.5
+    License: BSD 3-clause  (http://opensource.org/licenses/BSD-3-Clause)
+
+From: 'org.scalactic'
+  - scalactic (http://www.scalatest.org) org.scalactic:scalactic_2.12:bundle:3.0.1
+    License: the Apache License, ASL Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0)
+
+From: 'org.scalariform'
+  - scalariform (http://github.com/mdr/scalariform) org.scalariform:scalariform_2.11:jar:0.1.7
+    License: MIT License  (http://www.opensource.org/licenses/mit-license.php)
+
+From: 'org.scalastyle'
+  - scalastyle (http://www.scalastyle.org) org.scalastyle:scalastyle_2.11:jar:0.8.0
+    License: Apache-2.0  (http://www.apache.org/licenses/LICENSE-2.0.html)
+
+From: 'org.scalatest'
+  - scalatest (http://www.scalatest.org) org.scalatest:scalatest_2.12:bundle:3.0.1
+    License: the Apache License, ASL Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0)
+
+From: 'QOS.ch' (http://www.qos.ch)
+  - SLF4J API Module (http://www.slf4j.org) org.slf4j:slf4j-api:jar:1.7.25
+    License: MIT License  (http://www.opensource.org/licenses/mit-license.php)
+  - SLF4J Simple Binding (http://www.slf4j.org) org.slf4j:slf4j-simple:jar:1.7.25
+    License: MIT License  (http://www.opensource.org/licenses/mit-license.php)
+
+From: 'Scalastyle Organization' (https://github.com/scalastyle)
+  - Scalastyle Maven Plugin (http://www.scalastyle.org) org.scalastyle:scalastyle-maven-plugin:maven-plugin:0.8.0
+    License: Apache 2.0  (http://www.apache.org/licenses/LICENSE-2.0.html)
+
+From: 'Sonatype, Inc.' (http://www.sonatype.com)
+  - Sisu - Inject (JSR330 bean support) (http://sisu.sonatype.org/sisu-inject/guice-bean/sisu-inject-bean/) org.sonatype.sisu:sisu-inject-bean:bundle:1.4.2
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+  - Sisu - Inject (Plexus bean support) (http://sisu.sonatype.org/sisu-inject/guice-bean/guice-plexus/sisu-inject-plexus/) org.sonatype.sisu:sisu-inject-plexus:bundle:1.4.2
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+
+From: 'The Apache Software Foundation' (http://www.apache.org/)
+  - Apache Commons Codec (http://commons.apache.org/proper/commons-codec/) commons-codec:commons-codec:jar:1.9
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+  - Apache Commons Logging (http://commons.apache.org/proper/commons-logging/) commons-logging:commons-logging:jar:1.2
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+  - Apache HttpClient (http://hc.apache.org/httpcomponents-client) org.apache.httpcomponents:httpclient:jar:4.5.3
+    License: Apache License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+  - Apache HttpCore (http://hc.apache.org/httpcomponents-core-ga) org.apache.httpcomponents:httpcore:jar:4.4.6
+    License: Apache License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+  - Maven Artifact (http://maven.apache.org/maven-artifact/) org.apache.maven:maven-artifact:jar:3.0
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+  - Maven Model (http://maven.apache.org/maven-model/) org.apache.maven:maven-model:jar:3.0
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+  - Maven Plugin API (http://maven.apache.org/maven-plugin-api/) org.apache.maven:maven-plugin-api:jar:3.0
+    License: The Apache Software License, Version 2.0  (http://www.apache.org/licenses/LICENSE-2.0.txt)
+
+From: 'The Codehaus' (http://codehaus.org/)
+  - classworlds (http://classworlds.codehaus.org/) classworlds:classworlds:jar:1.1-alpha-2
+
+
+
+
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/META-INF/LICENSE
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/META-INF/LICENSE b/content/WEB-INF/classes/META-INF/LICENSE
new file mode 100644
index 0000000..d645695
--- /dev/null
+++ b/content/WEB-INF/classes/META-INF/LICENSE
@@ -0,0 +1,202 @@
+
+                                 Apache License
+                           Version 2.0, January 2004
+                        http://www.apache.org/licenses/
+
+   TERMS AND CONDITIONS FOR USE, REPRODUCTION, AND DISTRIBUTION
+
+   1. Definitions.
+
+      "License" shall mean the terms and conditions for use, reproduction,
+      and distribution as defined by Sections 1 through 9 of this document.
+
+      "Licensor" shall mean the copyright owner or entity authorized by
+      the copyright owner that is granting the License.
+
+      "Legal Entity" shall mean the union of the acting entity and all
+      other entities that control, are controlled by, or are under common
+      control with that entity. For the purposes of this definition,
+      "control" means (i) the power, direct or indirect, to cause the
+      direction or management of such entity, whether by contract or
+      otherwise, or (ii) ownership of fifty percent (50%) or more of the
+      outstanding shares, or (iii) beneficial ownership of such entity.
+
+      "You" (or "Your") shall mean an individual or Legal Entity
+      exercising permissions granted by this License.
+
+      "Source" form shall mean the preferred form for making modifications,
+      including but not limited to software source code, documentation
+      source, and configuration files.
+
+      "Object" form shall mean any form resulting from mechanical
+      transformation or translation of a Source form, including but
+      not limited to compiled object code, generated documentation,
+      and conversions to other media types.
+
+      "Work" shall mean the work of authorship, whether in Source or
+      Object form, made available under the License, as indicated by a
+      copyright notice that is included in or attached to the work
+      (an example is provided in the Appendix below).
+
+      "Derivative Works" shall mean any work, whether in Source or Object
+      form, that is based on (or derived from) the Work and for which the
+      editorial revisions, annotations, elaborations, or other modifications
+      represent, as a whole, an original work of authorship. For the purposes
+      of this License, Derivative Works shall not include works that remain
+      separable from, or merely link (or bind by name) to the interfaces of,
+      the Work and Derivative Works thereof.
+
+      "Contribution" shall mean any work of authorship, including
+      the original version of the Work and any modifications or additions
+      to that Work or Derivative Works thereof, that is intentionally
+      submitted to Licensor for inclusion in the Work by the copyright owner
+      or by an individual or Legal Entity authorized to submit on behalf of
+      the copyright owner. For the purposes of this definition, "submitted"
+      means any form of electronic, verbal, or written communication sent
+      to the Licensor or its representatives, including but not limited to
+      communication on electronic mailing lists, source code control systems,
+      and issue tracking systems that are managed by, or on behalf of, the
+      Licensor for the purpose of discussing and improving the Work, but
+      excluding communication that is conspicuously marked or otherwise
+      designated in writing by the copyright owner as "Not a Contribution."
+
+      "Contributor" shall mean Licensor and any individual or Legal Entity
+      on behalf of whom a Contribution has been received by Licensor and
+      subsequently incorporated within the Work.
+
+   2. Grant of Copyright License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      copyright license to reproduce, prepare Derivative Works of,
+      publicly display, publicly perform, sublicense, and distribute the
+      Work and such Derivative Works in Source or Object form.
+
+   3. Grant of Patent License. Subject to the terms and conditions of
+      this License, each Contributor hereby grants to You a perpetual,
+      worldwide, non-exclusive, no-charge, royalty-free, irrevocable
+      (except as stated in this section) patent license to make, have made,
+      use, offer to sell, sell, import, and otherwise transfer the Work,
+      where such license applies only to those patent claims licensable
+      by such Contributor that are necessarily infringed by their
+      Contribution(s) alone or by combination of their Contribution(s)
+      with the Work to which such Contribution(s) was submitted. If You
+      institute patent litigation against any entity (including a
+      cross-claim or counterclaim in a lawsuit) alleging that the Work
+      or a Contribution incorporated within the Work constitutes direct
+      or contributory patent infringement, then any patent licenses
+      granted to You under this License for that Work shall terminate
+      as of the date such litigation is filed.
+
+   4. Redistribution. You may reproduce and distribute copies of the
+      Work or Derivative Works thereof in any medium, with or without
+      modifications, and in Source or Object form, provided that You
+      meet the following conditions:
+
+      (a) You must give any other recipients of the Work or
+          Derivative Works a copy of this License; and
+
+      (b) You must cause any modified files to carry prominent notices
+          stating that You changed the files; and
+
+      (c) You must retain, in the Source form of any Derivative Works
+          that You distribute, all copyright, patent, trademark, and
+          attribution notices from the Source form of the Work,
+          excluding those notices that do not pertain to any part of
+          the Derivative Works; and
+
+      (d) If the Work includes a "NOTICE" text file as part of its
+          distribution, then any Derivative Works that You distribute must
+          include a readable copy of the attribution notices contained
+          within such NOTICE file, excluding those notices that do not
+          pertain to any part of the Derivative Works, in at least one
+          of the following places: within a NOTICE text file distributed
+          as part of the Derivative Works; within the Source form or
+          documentation, if provided along with the Derivative Works; or,
+          within a display generated by the Derivative Works, if and
+          wherever such third-party notices normally appear. The contents
+          of the NOTICE file are for informational purposes only and
+          do not modify the License. You may add Your own attribution
+          notices within Derivative Works that You distribute, alongside
+          or as an addendum to the NOTICE text from the Work, provided
+          that such additional attribution notices cannot be construed
+          as modifying the License.
+
+      You may add Your own copyright statement to Your modifications and
+      may provide additional or different license terms and conditions
+      for use, reproduction, or distribution of Your modifications, or
+      for any such Derivative Works as a whole, provided Your use,
+      reproduction, and distribution of the Work otherwise complies with
+      the conditions stated in this License.
+
+   5. Submission of Contributions. Unless You explicitly state otherwise,
+      any Contribution intentionally submitted for inclusion in the Work
+      by You to the Licensor shall be under the terms and conditions of
+      this License, without any additional terms or conditions.
+      Notwithstanding the above, nothing herein shall supersede or modify
+      the terms of any separate license agreement you may have executed
+      with Licensor regarding such Contributions.
+
+   6. Trademarks. This License does not grant permission to use the trade
+      names, trademarks, service marks, or product names of the Licensor,
+      except as required for reasonable and customary use in describing the
+      origin of the Work and reproducing the content of the NOTICE file.
+
+   7. Disclaimer of Warranty. Unless required by applicable law or
+      agreed to in writing, Licensor provides the Work (and each
+      Contributor provides its Contributions) on an "AS IS" BASIS,
+      WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or
+      implied, including, without limitation, any warranties or conditions
+      of TITLE, NON-INFRINGEMENT, MERCHANTABILITY, or FITNESS FOR A
+      PARTICULAR PURPOSE. You are solely responsible for determining the
+      appropriateness of using or redistributing the Work and assume any
+      risks associated with Your exercise of permissions under this License.
+
+   8. Limitation of Liability. In no event and under no legal theory,
+      whether in tort (including negligence), contract, or otherwise,
+      unless required by applicable law (such as deliberate and grossly
+      negligent acts) or agreed to in writing, shall any Contributor be
+      liable to You for damages, including any direct, indirect, special,
+      incidental, or consequential damages of any character arising as a
+      result of this License or out of the use or inability to use the
+      Work (including but not limited to damages for loss of goodwill,
+      work stoppage, computer failure or malfunction, or any and all
+      other commercial damages or losses), even if such Contributor
+      has been advised of the possibility of such damages.
+
+   9. Accepting Warranty or Additional Liability. While redistributing
+      the Work or Derivative Works thereof, You may choose to offer,
+      and charge a fee for, acceptance of support, warranty, indemnity,
+      or other liability obligations and/or rights consistent with this
+      License. However, in accepting such obligations, You may act only
+      on Your own behalf and on Your sole responsibility, not on behalf
+      of any other Contributor, and only if You agree to indemnify,
+      defend, and hold each Contributor harmless for any liability
+      incurred by, or claims asserted against, such Contributor by reason
+      of your accepting any such warranty or additional liability.
+
+   END OF TERMS AND CONDITIONS
+
+   APPENDIX: How to apply the Apache License to your work.
+
+      To apply the Apache License to your work, attach the following
+      boilerplate notice, with the fields enclosed by brackets "[]"
+      replaced with your own identifying information. (Don't include
+      the brackets!)  The text should be enclosed in the appropriate
+      comment syntax for the file format. We also recommend that a
+      file or class name and description of purpose be included on the
+      same "printed page" as the copyright notice for easier
+      identification within third-party archives.
+
+   Copyright [yyyy] [name of copyright owner]
+
+   Licensed 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.

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/META-INF/NOTICE
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/META-INF/NOTICE b/content/WEB-INF/classes/META-INF/NOTICE
new file mode 100644
index 0000000..65baee6
--- /dev/null
+++ b/content/WEB-INF/classes/META-INF/NOTICE
@@ -0,0 +1,8 @@
+
+Apache CarbonData :: Website
+Copyright 2017 The Apache Software Foundation
+
+This product includes software developed at
+The Apache Software Foundation (http://www.apache.org/).
+
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/MdFileHandler.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/MdFileHandler.class b/content/WEB-INF/classes/MdFileHandler.class
new file mode 100644
index 0000000..58088b3
Binary files /dev/null and b/content/WEB-INF/classes/MdFileHandler.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/Module.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/Module.class b/content/WEB-INF/classes/Module.class
new file mode 100644
index 0000000..a0bcb7b
Binary files /dev/null and b/content/WEB-INF/classes/Module.class differ

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

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/WebsiteLauncher.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/WebsiteLauncher.class b/content/WEB-INF/classes/WebsiteLauncher.class
new file mode 100644
index 0000000..52163dd
Binary files /dev/null and b/content/WEB-INF/classes/WebsiteLauncher.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/application.conf
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/application.conf b/content/WEB-INF/classes/application.conf
new file mode 100644
index 0000000..f7f5a09
--- /dev/null
+++ b/content/WEB-INF/classes/application.conf
@@ -0,0 +1,20 @@
+fileList=["configuration-parameters",
+  "ddl-operation-on-carbondata","quick-start-guide",
+  "dml-operation-on-carbondata",
+  "data-management",
+  "faq",
+  "file-structure-of-carbondata",
+  "installation-guide",
+  "supported-data-types-in-carbondata",
+  "troubleshooting",
+  "useful-tips-on-carbondata"
+  ]
+fileListToRetain=["quick-start-guide",
+  "installation-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/"
+mdLink="https://api.github.com/markdown/raw"
+mdFileLocation="src/site/markdown/"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/services/ConfService.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/services/ConfService.class b/content/WEB-INF/classes/services/ConfService.class
new file mode 100644
index 0000000..f38dfca
Binary files /dev/null and b/content/WEB-INF/classes/services/ConfService.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/services/ConfServiceImpl.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/services/ConfServiceImpl.class b/content/WEB-INF/classes/services/ConfServiceImpl.class
new file mode 100644
index 0000000..ca1af57
Binary files /dev/null and b/content/WEB-INF/classes/services/ConfServiceImpl.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/services/DataService.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/services/DataService.class b/content/WEB-INF/classes/services/DataService.class
new file mode 100644
index 0000000..4783e0d
Binary files /dev/null and b/content/WEB-INF/classes/services/DataService.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/services/DataServiceImpl.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/services/DataServiceImpl.class b/content/WEB-INF/classes/services/DataServiceImpl.class
new file mode 100644
index 0000000..f99cef9
Binary files /dev/null and b/content/WEB-INF/classes/services/DataServiceImpl.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/services/FileService.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/services/FileService.class b/content/WEB-INF/classes/services/FileService.class
new file mode 100644
index 0000000..ddd46db
Binary files /dev/null and b/content/WEB-INF/classes/services/FileService.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/classes/services/FileServiceImpl.class
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/services/FileServiceImpl.class b/content/WEB-INF/classes/services/FileServiceImpl.class
new file mode 100644
index 0000000..0dadcf9
Binary files /dev/null and b/content/WEB-INF/classes/services/FileServiceImpl.class differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/aopalliance-1.0.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/aopalliance-1.0.jar b/content/WEB-INF/lib/aopalliance-1.0.jar
new file mode 100644
index 0000000..578b1a0
Binary files /dev/null and b/content/WEB-INF/lib/aopalliance-1.0.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/classworlds-1.1-alpha-2.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/classworlds-1.1-alpha-2.jar b/content/WEB-INF/lib/classworlds-1.1-alpha-2.jar
new file mode 100644
index 0000000..1208b48
Binary files /dev/null and b/content/WEB-INF/lib/classworlds-1.1-alpha-2.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/commons-codec-1.9.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/commons-codec-1.9.jar b/content/WEB-INF/lib/commons-codec-1.9.jar
new file mode 100644
index 0000000..ef35f1c
Binary files /dev/null and b/content/WEB-INF/lib/commons-codec-1.9.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/commons-logging-1.2.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/commons-logging-1.2.jar b/content/WEB-INF/lib/commons-logging-1.2.jar
new file mode 100644
index 0000000..93a3b9f
Binary files /dev/null and b/content/WEB-INF/lib/commons-logging-1.2.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/config-1.3.1.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/config-1.3.1.jar b/content/WEB-INF/lib/config-1.3.1.jar
new file mode 100644
index 0000000..a21f42b
Binary files /dev/null and b/content/WEB-INF/lib/config-1.3.1.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/guice-3.0.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/guice-3.0.jar b/content/WEB-INF/lib/guice-3.0.jar
new file mode 100644
index 0000000..f313e2b
Binary files /dev/null and b/content/WEB-INF/lib/guice-3.0.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/httpclient-4.5.3.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/httpclient-4.5.3.jar b/content/WEB-INF/lib/httpclient-4.5.3.jar
new file mode 100644
index 0000000..8af4561
Binary files /dev/null and b/content/WEB-INF/lib/httpclient-4.5.3.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/httpcore-4.4.6.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/httpcore-4.4.6.jar b/content/WEB-INF/lib/httpcore-4.4.6.jar
new file mode 100644
index 0000000..16ed0d1
Binary files /dev/null and b/content/WEB-INF/lib/httpcore-4.4.6.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/javax.inject-1.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/javax.inject-1.jar b/content/WEB-INF/lib/javax.inject-1.jar
new file mode 100644
index 0000000..b2a9d0b
Binary files /dev/null and b/content/WEB-INF/lib/javax.inject-1.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/junit-3.8.1.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/junit-3.8.1.jar b/content/WEB-INF/lib/junit-3.8.1.jar
new file mode 100644
index 0000000..674d71e
Binary files /dev/null and b/content/WEB-INF/lib/junit-3.8.1.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/maven-artifact-3.0.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/maven-artifact-3.0.jar b/content/WEB-INF/lib/maven-artifact-3.0.jar
new file mode 100644
index 0000000..824aec5
Binary files /dev/null and b/content/WEB-INF/lib/maven-artifact-3.0.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/maven-model-3.0.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/maven-model-3.0.jar b/content/WEB-INF/lib/maven-model-3.0.jar
new file mode 100644
index 0000000..7993699
Binary files /dev/null and b/content/WEB-INF/lib/maven-model-3.0.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/maven-plugin-api-3.0.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/maven-plugin-api-3.0.jar b/content/WEB-INF/lib/maven-plugin-api-3.0.jar
new file mode 100644
index 0000000..bbb4f00
Binary files /dev/null and b/content/WEB-INF/lib/maven-plugin-api-3.0.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/mockito-all-1.9.5.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/mockito-all-1.9.5.jar b/content/WEB-INF/lib/mockito-all-1.9.5.jar
new file mode 100644
index 0000000..00416eb
Binary files /dev/null and b/content/WEB-INF/lib/mockito-all-1.9.5.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/plexus-classworlds-2.2.3.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/plexus-classworlds-2.2.3.jar b/content/WEB-INF/lib/plexus-classworlds-2.2.3.jar
new file mode 100644
index 0000000..182399c
Binary files /dev/null and b/content/WEB-INF/lib/plexus-classworlds-2.2.3.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/plexus-component-annotations-1.5.4.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/plexus-component-annotations-1.5.4.jar b/content/WEB-INF/lib/plexus-component-annotations-1.5.4.jar
new file mode 100644
index 0000000..bfb4895
Binary files /dev/null and b/content/WEB-INF/lib/plexus-component-annotations-1.5.4.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/plexus-container-default-1.0-alpha-9-stable-1.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/plexus-container-default-1.0-alpha-9-stable-1.jar b/content/WEB-INF/lib/plexus-container-default-1.0-alpha-9-stable-1.jar
new file mode 100644
index 0000000..d205236
Binary files /dev/null and b/content/WEB-INF/lib/plexus-container-default-1.0-alpha-9-stable-1.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/plexus-resources-1.0-alpha-7.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/plexus-resources-1.0-alpha-7.jar b/content/WEB-INF/lib/plexus-resources-1.0-alpha-7.jar
new file mode 100644
index 0000000..0e0b782
Binary files /dev/null and b/content/WEB-INF/lib/plexus-resources-1.0-alpha-7.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/plexus-utils-1.5.5.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/plexus-utils-1.5.5.jar b/content/WEB-INF/lib/plexus-utils-1.5.5.jar
new file mode 100644
index 0000000..ec865ba
Binary files /dev/null and b/content/WEB-INF/lib/plexus-utils-1.5.5.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/scala-library-2.12.1.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/scala-library-2.12.1.jar b/content/WEB-INF/lib/scala-library-2.12.1.jar
new file mode 100644
index 0000000..5396741
Binary files /dev/null and b/content/WEB-INF/lib/scala-library-2.12.1.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/scala-parser-combinators_2.11-1.0.1.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/scala-parser-combinators_2.11-1.0.1.jar b/content/WEB-INF/lib/scala-parser-combinators_2.11-1.0.1.jar
new file mode 100644
index 0000000..fd758c8
Binary files /dev/null and b/content/WEB-INF/lib/scala-parser-combinators_2.11-1.0.1.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/scala-parser-combinators_2.12-1.0.4.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/scala-parser-combinators_2.12-1.0.4.jar b/content/WEB-INF/lib/scala-parser-combinators_2.12-1.0.4.jar
new file mode 100644
index 0000000..56fa393
Binary files /dev/null and b/content/WEB-INF/lib/scala-parser-combinators_2.12-1.0.4.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/scala-reflect-2.12.0.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/scala-reflect-2.12.0.jar b/content/WEB-INF/lib/scala-reflect-2.12.0.jar
new file mode 100644
index 0000000..bf7a29a
Binary files /dev/null and b/content/WEB-INF/lib/scala-reflect-2.12.0.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/scala-xml_2.11-1.0.1.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/scala-xml_2.11-1.0.1.jar b/content/WEB-INF/lib/scala-xml_2.11-1.0.1.jar
new file mode 100644
index 0000000..7a595de
Binary files /dev/null and b/content/WEB-INF/lib/scala-xml_2.11-1.0.1.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/scala-xml_2.12-1.0.5.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/scala-xml_2.12-1.0.5.jar b/content/WEB-INF/lib/scala-xml_2.12-1.0.5.jar
new file mode 100644
index 0000000..43cba8a
Binary files /dev/null and b/content/WEB-INF/lib/scala-xml_2.12-1.0.5.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/scalactic_2.12-3.0.1.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/scalactic_2.12-3.0.1.jar b/content/WEB-INF/lib/scalactic_2.12-3.0.1.jar
new file mode 100644
index 0000000..0526f24
Binary files /dev/null and b/content/WEB-INF/lib/scalactic_2.12-3.0.1.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/scalariform_2.11-0.1.7.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/scalariform_2.11-0.1.7.jar b/content/WEB-INF/lib/scalariform_2.11-0.1.7.jar
new file mode 100644
index 0000000..04cad4d
Binary files /dev/null and b/content/WEB-INF/lib/scalariform_2.11-0.1.7.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/scalastyle-maven-plugin-0.8.0.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/scalastyle-maven-plugin-0.8.0.jar b/content/WEB-INF/lib/scalastyle-maven-plugin-0.8.0.jar
new file mode 100644
index 0000000..cc9df99
Binary files /dev/null and b/content/WEB-INF/lib/scalastyle-maven-plugin-0.8.0.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/scalastyle_2.11-0.8.0.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/scalastyle_2.11-0.8.0.jar b/content/WEB-INF/lib/scalastyle_2.11-0.8.0.jar
new file mode 100644
index 0000000..0acd99f
Binary files /dev/null and b/content/WEB-INF/lib/scalastyle_2.11-0.8.0.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/scalatest_2.12-3.0.1.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/scalatest_2.12-3.0.1.jar b/content/WEB-INF/lib/scalatest_2.12-3.0.1.jar
new file mode 100644
index 0000000..1fa54ea
Binary files /dev/null and b/content/WEB-INF/lib/scalatest_2.12-3.0.1.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/sisu-guice-2.1.7-noaop.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/sisu-guice-2.1.7-noaop.jar b/content/WEB-INF/lib/sisu-guice-2.1.7-noaop.jar
new file mode 100644
index 0000000..aa66844
Binary files /dev/null and b/content/WEB-INF/lib/sisu-guice-2.1.7-noaop.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/sisu-inject-bean-1.4.2.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/sisu-inject-bean-1.4.2.jar b/content/WEB-INF/lib/sisu-inject-bean-1.4.2.jar
new file mode 100644
index 0000000..ca6abdd
Binary files /dev/null and b/content/WEB-INF/lib/sisu-inject-bean-1.4.2.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/sisu-inject-plexus-1.4.2.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/sisu-inject-plexus-1.4.2.jar b/content/WEB-INF/lib/sisu-inject-plexus-1.4.2.jar
new file mode 100644
index 0000000..53063c7
Binary files /dev/null and b/content/WEB-INF/lib/sisu-inject-plexus-1.4.2.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/slf4j-api-1.7.25.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/slf4j-api-1.7.25.jar b/content/WEB-INF/lib/slf4j-api-1.7.25.jar
new file mode 100644
index 0000000..0143c09
Binary files /dev/null and b/content/WEB-INF/lib/slf4j-api-1.7.25.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/WEB-INF/lib/slf4j-simple-1.7.25.jar
----------------------------------------------------------------------
diff --git a/content/WEB-INF/lib/slf4j-simple-1.7.25.jar b/content/WEB-INF/lib/slf4j-simple-1.7.25.jar
new file mode 100644
index 0000000..a7260f3
Binary files /dev/null and b/content/WEB-INF/lib/slf4j-simple-1.7.25.jar differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/20e101ec/content/pdf/maven-pdf-plugin.pdf
----------------------------------------------------------------------
diff --git a/content/pdf/maven-pdf-plugin.pdf b/content/pdf/maven-pdf-plugin.pdf
new file mode 100644
index 0000000..290e323
Binary files /dev/null and b/content/pdf/maven-pdf-plugin.pdf differ


[8/9] carbondata-site git commit: Added functionality to auto generate PDF for Documentation

Posted by ch...@apache.org.
Added functionality to auto generate PDF for Documentation


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

Branch: refs/heads/asf-site
Commit: 3d4c33e0bfc5bcc2edf42cd53ef9dbb0d45e14c2
Parents: cf9ebc6 dd843c6
Author: chenliang613 <ch...@apache.org>
Authored: Wed May 24 12:16:30 2017 +0800
Committer: chenliang613 <ch...@apache.org>
Committed: Wed May 24 12:16:30 2017 +0800

----------------------------------------------------------------------
 carbonscript.sh                                 |   4 +
 content/pdf/maven-pdf-plugin.pdf                | Bin 145272 -> 0 bytes
 pom.xml                                         | 124 ++++-
 src/main/resources/application.conf             |   3 +-
 src/main/scala/CleanUp.scala                    |  26 +
 src/main/scala/MDFileConverter.scala            |  20 +-
 src/main/scala/WebsiteLauncher.scala            |  23 +-
 src/main/webapp/pdf/maven-pdf-plugin.pdf        | Bin 145272 -> 193057 bytes
 src/site/images/CarbonData_logo.png             | Bin 0 -> 10971 bytes
 .../images/carbon_data_file_structure_new.png   | Bin 0 -> 9477 bytes
 src/site/images/carbon_data_format_new.png      | Bin 0 -> 35510 bytes
 src/site/pdf.xml                                |  35 ++
 src/site/projectLogo/ApacheLogo.png             | Bin 0 -> 44839 bytes
 src/site/projectLogo/CarbonDataLogo.png         | Bin 0 -> 6844 bytes
 src/site/resources/pdf-config.xml               | 520 +++++++++++++++++++
 src/test/scala/MDFileConverterSpec.scala        |   2 +-
 16 files changed, 738 insertions(+), 19 deletions(-)
----------------------------------------------------------------------



[7/9] carbondata-site git commit: Integrated auto generate PDF Functionality

Posted by ch...@apache.org.
Integrated auto generate PDF Functionality


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

Branch: refs/heads/asf-site
Commit: dd843c69f00b05a73fcdb4fbe24bebe4fe26af8e
Parents: 3a8c7ff
Author: PallaviSingh1992 <pa...@yahoo.co.in>
Authored: Wed May 24 00:26:07 2017 +0530
Committer: PallaviSingh1992 <pa...@yahoo.co.in>
Committed: Wed May 24 00:26:07 2017 +0530

----------------------------------------------------------------------
 carbonscript.sh                          |   4 +-
 pom.xml                                  |  73 ++++++++++++++++----------
 src/main/scala/CleanUp.scala             |  26 +++++++++
 src/main/scala/MDFileConverter.scala     |  21 +++++++-
 src/main/scala/WebsiteLauncher.scala     |  23 ++++++--
 src/main/webapp/pdf/maven-pdf-plugin.pdf | Bin 145272 -> 193057 bytes
 src/site/projectLogo/ApacheLogo.png      | Bin 7241 -> 44839 bytes
 src/site/projectLogo/CarbonDataLogo.png  | Bin 10971 -> 6844 bytes
 src/test/scala/MDFileConverterSpec.scala |   2 +-
 9 files changed, 111 insertions(+), 38 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/dd843c69/carbonscript.sh
----------------------------------------------------------------------
diff --git a/carbonscript.sh b/carbonscript.sh
index 538922d..31453e4 100755
--- a/carbonscript.sh
+++ b/carbonscript.sh
@@ -2,6 +2,8 @@ mvn compile
 
 mvn exec:java -Dexec.mainClass=WebsiteLauncher
 
+mvn pdf:pdf
+
 mvn install
 
-mvn pdf:pdf
\ No newline at end of file
+mvn exec:java -Dexec.mainClass=CleanUp
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/dd843c69/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index a863151..3ebdd73 100644
--- a/pom.xml
+++ b/pom.xml
@@ -56,16 +56,31 @@
               </execution>
           </executions>
       </plugin>
-
-      <plugin>
-        <groupId>org.codehaus.mojo</groupId>
-        <artifactId>exec-maven-plugin</artifactId>
-        <version>1.2.1</version>
-        <configuration>
-          <mainClass>WebsiteLauncher</mainClass>
-        </configuration>
-      </plugin>
-
+        <plugin>
+            <groupId>org.codehaus.mojo</groupId>
+            <artifactId>exec-maven-plugin</artifactId>
+            <version>1.2.1</version>
+            <executions>
+                <execution>
+                    <id>first-execution</id>
+                    <goals>
+                        <goal>java</goal>
+                    </goals>
+                    <configuration>
+                        <mainClass>WebsiteLauncher</mainClass>
+                    </configuration>
+                </execution>
+                <execution>
+                    <id>second-execution</id>
+                    <goals>
+                        <goal>java</goal>
+                    </goals>
+                    <configuration>
+                        <mainClass>CleanUp</mainClass>
+                    </configuration>
+                </execution>
+            </executions>
+        </plugin>
       <plugin>
         <groupId>org.mortbay.jetty</groupId>
         <artifactId>jetty-maven-plugin</artifactId>
@@ -173,25 +188,6 @@
                     </configuration>
                 </execution>
                 <execution>
-                    <id>publish-pdf</id>
-                    <phase>install</phase>
-                    <goals>
-                        <goal>copy-resources</goal>
-                    </goals>
-                    <configuration>
-                        <outputDirectory>${basedir}/src/pdf</outputDirectory>
-                        <resources>
-                            <resource>
-                                <directory>${basedir}/target/pdf</directory>
-                                <filtering>false</filtering>
-                                <includes>
-                                    <include>**/*.pdf</include>
-                                </includes>
-                            </resource>
-                        </resources>
-                    </configuration>
-                </execution>
-                <execution>
                     <id>copy-pdf</id>
                     <phase>install</phase>
                     <goals>
@@ -210,6 +206,25 @@
                         </resources>
                     </configuration>
                 </execution>
+                <execution>
+                   <id>publish-pdf</id>
+                   <phase>install</phase>
+                   <goals>
+                       <goal>copy-resources</goal>
+                   </goals>
+                   <configuration>
+                       <outputDirectory>${basedir}/src/main/webapp/pdf</outputDirectory>
+                       <resources>
+                           <resource>
+                               <directory>${basedir}/target/pdf</directory>
+                               <filtering>false</filtering>
+                               <includes>
+                                   <include>**/*.pdf</include>
+                               </includes>
+                           </resource>
+                       </resources>
+                   </configuration>
+               </execution>
             </executions>
         </plugin>
     </plugins>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/dd843c69/src/main/scala/CleanUp.scala
----------------------------------------------------------------------
diff --git a/src/main/scala/CleanUp.scala b/src/main/scala/CleanUp.scala
new file mode 100644
index 0000000..de6bf0c
--- /dev/null
+++ b/src/main/scala/CleanUp.scala
@@ -0,0 +1,26 @@
+import java.io.File
+
+import org.slf4j.{Logger, LoggerFactory}
+import services.ConfServiceImpl
+
+object CleanUp {
+  def main(args: Array[String]): Unit = {
+
+    val logger: Logger = LoggerFactory.getLogger("CleanUp")
+
+    val conf = new ConfServiceImpl
+
+    logger.info("Starting clean up after PDF Generation")
+    cleanUp(init(conf.readString("mdFileLocation")))
+    logger.info("Successfully completed clean up after PDF Generation")
+  }
+
+  private def init(path: String): File = {
+    new File(path)
+  }
+
+  private def cleanUp(file: File): Boolean = {
+    file.listFiles.map(file => file.delete())
+    file.delete()
+  }
+}

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/dd843c69/src/main/scala/MDFileConverter.scala
----------------------------------------------------------------------
diff --git a/src/main/scala/MDFileConverter.scala b/src/main/scala/MDFileConverter.scala
index 4fab013..bc5a6f8 100644
--- a/src/main/scala/MDFileConverter.scala
+++ b/src/main/scala/MDFileConverter.scala
@@ -18,7 +18,7 @@ class MDFileConverter @Inject()(fileService: FileService, confService: ConfServi
     *
     * @return status of each file i.e. success or failure
     */
-  def convertToHtml(): String = {
+  def convertToHtml(status: Boolean): String = {
     val listOfFiles: List[String] = confService.readListOfString("fileList")
     val statusList: List[String] = listOfFiles.map { file =>
       val fileURLContent: String = dataService.dataOnGetRequest(url + file + inputFileExtension)
@@ -27,7 +27,7 @@ class MDFileConverter @Inject()(fileService: FileService, confService: ConfServi
         case Some(data: String) => val fileData = fileReadObject.convertMdExtensions(data)
           logger.info(s"Begin writing [ $file outputFileExtension ] at $location")
           val statusHtmlFile = fileService.writeToFile(location + file + outputFileExtension, headerContent + fileData + footerContent)
-          val statusMdFile = fileService.writeToFile(confService.readString("mdFileLocation") + file + inputFileExtension, fileURLContent)
+          saveMdFilesForPDF(status, fileURLContent, file)
           if (statusHtmlFile) {
             logger.info(s"Successfully written [ $file $outputFileExtension ] at $location")
             "Success"
@@ -50,5 +50,22 @@ class MDFileConverter @Inject()(fileService: FileService, confService: ConfServi
       "[SUCCESS]: All files converted successfully."
     }
   }
+
+  /**
+    * saves the fetched MD Files for generation of PDF
+    * @param status
+    * @param fileURLContent
+    * @param file
+    * @return
+    */
+  private def saveMdFilesForPDF(status: Boolean, fileURLContent: String, file: String): Boolean = {
+    if (status) {
+      logger.info(s"[SUCCESS] :Saving Markdown : $file for PDF Generation")
+      fileService.writeToFile(confService.readString("mdFileLocation") + file + inputFileExtension, fileURLContent)
+    } else {
+      logger.error(s"[ERROR] : Failed to save Markdown :$file for PDF Generation")
+      false
+    }
+  }
 }
 

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/dd843c69/src/main/scala/WebsiteLauncher.scala
----------------------------------------------------------------------
diff --git a/src/main/scala/WebsiteLauncher.scala b/src/main/scala/WebsiteLauncher.scala
index 7919987..f814769 100644
--- a/src/main/scala/WebsiteLauncher.scala
+++ b/src/main/scala/WebsiteLauncher.scala
@@ -1,13 +1,26 @@
+import java.io.File
+
 import org.slf4j.{Logger, LoggerFactory}
-import services.{ConfServiceImpl, FileServiceImpl, DataServiceImpl}
+import services.{ConfServiceImpl, DataServiceImpl, FileServiceImpl}
 
-object WebsiteLauncher  {
-  def main(args: Array[String]):Unit= {
+object WebsiteLauncher {
+  def main(args: Array[String]): Unit = {
     val logger: Logger = LoggerFactory.getLogger("FileOperationMain")
 
-    val fileObject: MDFileConverter = new MDFileConverter(new FileServiceImpl,new ConfServiceImpl,new DataServiceImpl)
+    val conf = new ConfServiceImpl
+
+    val file = init(conf.readString("mdFileLocation"))
+
+    val dirStatus = file.mkdir()
+
+    val fileObject: MDFileConverter = new MDFileConverter(new FileServiceImpl, new ConfServiceImpl, new DataServiceImpl)
 
-    val status: String = fileObject.convertToHtml()
+    val status: String = fileObject.convertToHtml(dirStatus)
     logger.info(s"File Conversion to html : $status")
   }
+
+  private def init(path: String): File = {
+    new File(path)
+  }
+
 }

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/dd843c69/src/main/webapp/pdf/maven-pdf-plugin.pdf
----------------------------------------------------------------------
diff --git a/src/main/webapp/pdf/maven-pdf-plugin.pdf b/src/main/webapp/pdf/maven-pdf-plugin.pdf
index 5f701eb..290e323 100644
Binary files a/src/main/webapp/pdf/maven-pdf-plugin.pdf and b/src/main/webapp/pdf/maven-pdf-plugin.pdf differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/dd843c69/src/site/projectLogo/ApacheLogo.png
----------------------------------------------------------------------
diff --git a/src/site/projectLogo/ApacheLogo.png b/src/site/projectLogo/ApacheLogo.png
index 9d25899..c25efca 100644
Binary files a/src/site/projectLogo/ApacheLogo.png and b/src/site/projectLogo/ApacheLogo.png differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/dd843c69/src/site/projectLogo/CarbonDataLogo.png
----------------------------------------------------------------------
diff --git a/src/site/projectLogo/CarbonDataLogo.png b/src/site/projectLogo/CarbonDataLogo.png
index bc09b23..c1e6f20 100644
Binary files a/src/site/projectLogo/CarbonDataLogo.png and b/src/site/projectLogo/CarbonDataLogo.png differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/dd843c69/src/test/scala/MDFileConverterSpec.scala
----------------------------------------------------------------------
diff --git a/src/test/scala/MDFileConverterSpec.scala b/src/test/scala/MDFileConverterSpec.scala
index 92d11e6..17f63c0 100644
--- a/src/test/scala/MDFileConverterSpec.scala
+++ b/src/test/scala/MDFileConverterSpec.scala
@@ -31,6 +31,6 @@ class MDFileConverterSpec extends FlatSpec with MockitoSugar {
   val fileModification = new MDFileConverter(fileServiceMock, confServiceMock, dataServiceMock)
 
   it should "return successful convert list of files to html files" in {
-    assert(fileModification.convertToHtml() === "[SUCCESS]: All files converted successfully.")
+    assert(fileModification.convertToHtml(true) === "[SUCCESS]: All files converted successfully.")
   }
 }


[3/9] carbondata-site git commit: pdf plugins added

Posted by ch...@apache.org.
pdf plugins added


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

Branch: refs/heads/asf-site
Commit: 88ecec59d6718a56e40e58bda0128e5bf560eb4d
Parents: 7a262a0
Author: jatin <ja...@knoldus.in>
Authored: Mon May 22 14:10:21 2017 +0530
Committer: jatin <ja...@knoldus.in>
Committed: Mon May 22 14:10:21 2017 +0530

----------------------------------------------------------------------
 pom.xml | 86 +++++++++++++++++++++++++++++++++++++++++++++++++++++++++++-
 1 file changed, 85 insertions(+), 1 deletion(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/88ecec59/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 8b1154e..0c1a1cd 100644
--- a/pom.xml
+++ b/pom.xml
@@ -125,6 +125,86 @@
                 </execution>
             </executions>
         </plugin>
+        <!--PDF Generation Plugin -->
+        <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-pdf-plugin</artifactId>
+            <version>1.3</version>
+            <executions>
+                <execution>
+                    <id>pdf</id>
+                    <phase>site</phase>
+                    <goals>
+                        <goal>pdf</goal>
+                    </goals>
+                    <configuration>
+                        <outputDirectory>${project.reporting.outputDirectory}</outputDirectory>
+                        <includeReports>false</includeReports>
+                    </configuration>
+                </execution>
+            </executions>
+        </plugin>
+        <plugin>
+            <groupId>org.apache.maven.plugins</groupId>
+            <artifactId>maven-resources-plugin</artifactId>
+            <version>3.0.2</version>
+            <executions>
+                <execution>
+                    <id>publish-pdf</id>
+                    <phase>install</phase>
+                    <goals>
+                        <goal>copy-resources</goal>
+                    </goals>
+                    <configuration>
+                        <outputDirectory>${basedir}/src/pdf</outputDirectory>
+                        <resources>
+                            <resource>
+                                <directory>${basedir}/target/pdf</directory>
+                                <filtering>false</filtering>
+                                <includes>
+                                    <include>**/*.pdf</include>
+                                </includes>
+                            </resource>
+                        </resources>
+                    </configuration>
+                </execution>
+                <execution>
+                    <id>publish-site</id>
+                    <phase>install</phase>
+                    <goals>
+                        <goal>copy-resources</goal>
+                    </goals>
+                    <configuration>
+                        <outputDirectory>${basedir}/content</outputDirectory>
+                        <resources>
+                            <resource>
+                                <directory>${basedir}/target/site-17</directory>
+                                <filtering>false</filtering>
+                            </resource>
+                        </resources>
+                    </configuration>
+                </execution>
+                <execution>
+                    <id>copy-pdf</id>
+                    <phase>install</phase>
+                    <goals>
+                        <goal>copy-resources</goal>
+                    </goals>
+                    <configuration>
+                        <outputDirectory>${basedir}/content/pdf/</outputDirectory>
+                        <resources>
+                            <resource>
+                                <directory>${basedir}/target/pdf</directory>
+                                <filtering>false</filtering>
+                                <includes>
+                                    <include>**/*.pdf</include>
+                                </includes>
+                            </resource>
+                        </resources>
+                    </configuration>
+                </execution>
+            </executions>
+        </plugin>
     </plugins>
   </build>
 
@@ -134,7 +214,11 @@
       <artifactId>scala-library</artifactId>
       <version>2.12.1</version>
     </dependency>
-
+      <dependency>
+          <groupId>org.apache.maven.doxia</groupId>
+          <artifactId>doxia-module-markdown</artifactId>
+          <version>1.6</version>
+      </dependency>
     <!-- https://mvnrepository.com/artifact/org.apache.httpcomponents/httpclient -->
     <dependency>
       <groupId>org.apache.httpcomponents</groupId>


[4/9] carbondata-site git commit: Added Functionality of PdfGeneration

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/site/markdown/troubleshooting.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/troubleshooting.md b/src/site/markdown/troubleshooting.md
deleted file mode 100644
index 9181d83..0000000
--- a/src/site/markdown/troubleshooting.md
+++ /dev/null
@@ -1,247 +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.
-
-## 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.
-
-## Failure to read the metastore db during table creation.
-
-  **Symptom**
-
-  We get the following exception on trying to connect :
-
-  ```
-  Cannot read the metastore db
-  ```
-
-  **Possible Cause**
-
-  The metastore db is dysfunctional.
-
-  **Procedure**
-
-  Remove the metastore db from the carbon.metastore in the Spark Directory.
-
-## Failed to load data on the cluster
-
-  **Symptom**
-
-  Data loading fails with the following exception :
-
-   ```
-   Data Load failure exeception
-   ```
-
-  **Possible Cause**
-
-  The following issue can cause the failure :
-
-  1. The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.
-
-  2. Path to hdfs ddl is not configured correctly in the carbon.properties.
-
-  **Procedure**
-
-   Follow the steps to ensure the following configuration files are consistent across all the nodes:
-
-   1. Copy the core-site.xml, hive-site.xml, yarn-site,carbon.properties files from the master node to all the other nodes in the cluster.
-      For example, you can use scp to copy this file to all the nodes.
-
-      Note : Set the path to hdfs ddl in carbon.properties in the master node.
-
-   2. For the changes to take effect, restart the Spark cluster.
-
-
-
-## Failed to insert data on the cluster
-
-  **Symptom**
-
-  Insertion fails with the following exception :
-
-   ```
-   Data Load failure exeception
-   ```
-
-  **Possible Cause**
-
-  The following issue can cause the failure :
-
-  1. The core-site.xml, hive-site.xml, yarn-site and carbon.properties are not consistent across all nodes of the cluster.
-
-  2. Path to hdfs ddl is not configured correctly in the carbon.properties.
-
-  **Procedure**
-
-   Follow the steps to ensure the following configuration files are consistent across all the nodes:
-
-   1. Copy the core-site.xml, hive-site.xml, yarn-site,carbon.properties files from the master node to all the other nodes in the cluster.
-      For example, you can use scp to copy this file to all the nodes.
-
-      Note : Set the path to hdfs ddl in carbon.properties in the master node.
-
-   2. For the changes to take effect, restart the Spark cluster.
-
-## Failed to execute Concurrent Operations(Load,Insert,Update) on table by multiple workers.
-
-  **Symptom**
-
-  Execution fails with the following exception :
-
-   ```
-   Table is locked for updation.
-   ```
-
-  **Possible Cause**
-
-  Concurrency not supported.
-
-  **Procedure**
-
-  Worker must wait for the query execution to complete and the table to release the lock for another query execution to succeed..
-
-## Failed to create a table with a single numeric column.
-
-  **Symptom**
-
-  Execution fails with the following exception :
-
-   ```
-   Table creation fails.
-   ```
-
-  **Possible Cause**
-
-  Behavior 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/069b05da/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 b1ff903..0000000
--- a/src/site/markdown/useful-tips-on-carbondata.md
+++ /dev/null
@@ -1,180 +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 above topics :
-
-* [Suggestions to create CarbonData Table](#suggestions-to-create-carbondata-table)
-* [Configurations For Optimizing CarbonData Performance](#configurations-for-optimizing-carbondata-performance)
-
-## Suggestions to Create CarbonData Table
-
-Recently CarbonData was used to analyze performance of Telecommunication field.
-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   | Numeric(20,0) | NA          | Measure     |
-| ...         | ...           | NA          | Measure     |
-| counter_100 | Numeric(20,0) | NA          | Measure     |
-
-CarbonData has more than 50 test cases, on the basis of these we have following suggestions to enhance the query performance :
-
-
-
-* **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,
-  ...
-  )STORED BY 'org.apache.carbondata.format' 
-  TBLPROPERTIES ( 'DICTIONARY_EXCLUDE'='MSISDN,..',
-  'DICTIONARY_INCLUDE'='...');
-```
-  
-  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 as Dime_1 has the lowest cardinality. 
-  The create table command can be modified as suggested below :
-
-```
-  create table carbondata_table(
-  Dime_1 String,
-  HOST String,
-  MSISDN String,
-  ...
-  )STORED BY 'org.apache.carbondata.format' 
-  TBLPROPERTIES ( 'DICTIONARY_EXCLUDE'='MSISDN,HOST..',
-  'DICTIONARY_INCLUDE'='Dime_1..');
-```
-
-
-* **Put the Dimension type columns in order of low to high cardinality**
-
-  If the columns used to filter are not frequently used, then it is suggested to order all the columns of dimension type in order of low to high cardinality.
-The create table command can be modified as below :
-
-```
-  create table carbondata_table(
-  Dime_1 String,
-  BEGIN_TIME bigint
-  HOST String,
-  MSISDN String,
-  ...
-  )STORED BY 'org.apache.carbondata.format' 
-  TBLPROPERTIES ( 'DICTIONARY_EXCLUDE'='MSISDN,HOST,IMSI..',
-  'DICTIONARY_INCLUDE'='Dime_1,END_TIME,BEGIN_TIME..');
-```
-
-
-* **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
-  HOST String,
-  MSISDN String,
-  counter_1 double,
-  counter_2 double,
-  ...
-  counter_100 double
-  )STORED BY 'org.apache.carbondata.format' 
-  TBLPROPERTIES ( 'DICTIONARY_EXCLUDE'='MSISDN,HOST,IMSI',
-  'DICTIONARY_INCLUDE'='Dime_1,END_TIME,BEGIN_TIME');
-```
-  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 start_time is incremental for each load, it is
-suggested to put start_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,
-  ...
-  counter_100 double
-  )STORED BY 'org.apache.carbondata.format' 
-  TBLPROPERTIES ( 'DICTIONARY_EXCLUDE'='MSISDN,HOST,IMSI',
-  'DICTIONARY_INCLUDE'='Dime_1,END_TIME,BEGIN_TIME'); 
-```
-
-
-* **Avoid adding high cardinality columns to dictionary**
-
-  If the system has low memory configuration, then it is suggested to exclude high cardinality columns from the dictionary to 
-enhance load performance. Creation of  dictionary for high cardinality columns at time of load will degrade load performance due to 
-excessive memory usage. 
-
-  By default CarbonData determines the cardinality at the first data load and allows for dictionary creation only if the cardinality is less than
-1 million.
-
-
-## 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/con/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. |
-| num-executors/executor-cores/executor-memory | spark/con/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. |
-
-
- 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/site/pdf.xml
----------------------------------------------------------------------
diff --git a/src/site/pdf.xml b/src/site/pdf.xml
new file mode 100644
index 0000000..b11b3cc
--- /dev/null
+++ b/src/site/pdf.xml
@@ -0,0 +1,35 @@
+<document xmlns="http://maven.apache.org/DOCUMENT/1.0.1"
+          xmlns:xsi="http://www.w3.org/2001/XMLSchema-instance"
+          xsi:schemaLocation="http://maven.apache.org/DOCUMENT/1.0.1 http://maven.apache.org/xsd/document-1.0.1.xsd"
+          outputName="maven-pdf-plugin">
+
+  <meta>
+    <title>CarbonData Documentation</title>
+    <author>The Apache CarbonData Community</author>
+  </meta>
+
+  <toc name="Table of Contents">
+     <item name="Quick Start" ref='quick-start-guide.md'/>
+   <item name="CarbonData File Structure" ref='file-structure-of-carbondata.md'/>
+    <item name="Data Types" ref='supported-data-types-in-carbondata.md'/>
+    <item name="Data Management" ref='data-management.md'/>
+      <item name="DDL" ref='ddl-operation-on-carbondata.md'/>
+        <item name="DML" ref='dml-operation-on-carbondata.md '/>
+        <item name="Installation" ref='installation-guide.md'/>
+    <item name="Configuring CarbonData" ref='configuration-parameters.md'/>
+    <item name="FAQs" ref='faq.md'/>
+    <item name="Troubleshooting" ref='troubleshooting.md'/>
+    <item name="Useful Tips" ref='useful-tips-on-carbondata.md'/>
+
+  </toc>
+
+  <cover>
+    <companyLogo>../../src/site/projectLogo/ApacheLogo.png</companyLogo>
+    <projectLogo>../../src/site/projectLogo/CarbonDataLogo.png</projectLogo>
+    <coverTitle>Apache CarbonData</coverTitle>
+    <coverSubTitle>Ver 1.0 </coverSubTitle>
+    <coverType>Documentation</coverType>
+    <projectName>Apache CarbonData</projectName>
+    <companyName>The Apache Software Foundation</companyName>
+  </cover>
+</document>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/site/projectLogo/ApacheLogo.png
----------------------------------------------------------------------
diff --git a/src/site/projectLogo/ApacheLogo.png b/src/site/projectLogo/ApacheLogo.png
new file mode 100644
index 0000000..9d25899
Binary files /dev/null and b/src/site/projectLogo/ApacheLogo.png differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/site/projectLogo/CarbonDataLogo.png
----------------------------------------------------------------------
diff --git a/src/site/projectLogo/CarbonDataLogo.png b/src/site/projectLogo/CarbonDataLogo.png
new file mode 100644
index 0000000..bc09b23
Binary files /dev/null and b/src/site/projectLogo/CarbonDataLogo.png differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/site/resources/pdf-config.xml
----------------------------------------------------------------------
diff --git a/src/site/resources/pdf-config.xml b/src/site/resources/pdf-config.xml
new file mode 100644
index 0000000..470baf8
--- /dev/null
+++ b/src/site/resources/pdf-config.xml
@@ -0,0 +1,520 @@
+<?xml version="1.0" encoding="UTF-8"?>
+<!--
+  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.
+-->
+
+<!-- These values are optimized for an A4 paper size. -->
+
+<xsl:stylesheet xmlns:xsl="http://www.w3.org/1999/XSL/Transform" xmlns:fo="http://www.w3.org/1999/XSL/Format" version="1.0">
+    <!--<fo:root xmlns:fo="http://www.w3.org/1999/XSL/Format"></fo:root>-->
+    <!-- Layout master sets -->
+    <xsl:attribute-set name="layout.master.set.base">
+        <xsl:attribute name="page-width">8.27in</xsl:attribute>
+        <xsl:attribute name="page-height">11.70in</xsl:attribute>
+        <xsl:attribute name="margin-top">0.625in</xsl:attribute>
+        <xsl:attribute name="margin-bottom">0.6in</xsl:attribute>
+        <xsl:attribute name="margin-left">1in</xsl:attribute>
+        <xsl:attribute name="margin-right">1in</xsl:attribute>
+    </xsl:attribute-set>
+
+    <xsl:attribute-set name="layout.master.set.cover-page" use-attribute-sets="layout.master.set.base">
+        <xsl:attribute name="master-name">cover-page</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="layout.master.set.cover-page.region-body">
+        <xsl:attribute name="margin-top">0.7in</xsl:attribute>
+    </xsl:attribute-set>
+
+    <xsl:attribute-set name="layout.master.set.cover-inside" use-attribute-sets="layout.master.set.base">
+        <xsl:attribute name="master-name">cover-inside</xsl:attribute>
+        <xsl:attribute name="margin-top">0in</xsl:attribute>
+        <xsl:attribute name="margin-bottom">0in</xsl:attribute>
+        <xsl:attribute name="margin-left">0in</xsl:attribute>
+        <xsl:attribute name="margin-right">0in</xsl:attribute>
+    </xsl:attribute-set>
+
+    <xsl:attribute-set name="layout.master.set.toc" use-attribute-sets="layout.master.set.base">
+        <xsl:attribute name="master-name">toc</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="layout.master.set.toc.region-body">
+        <xsl:attribute name="margin-top">0.7in</xsl:attribute>
+        <xsl:attribute name="margin-bottom">0.8in</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="layout.master.set.toc.region-before">
+        <xsl:attribute name="extent">0.35in</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="layout.master.set.toc.region-after">
+        <xsl:attribute name="extent">0.125in</xsl:attribute>
+    </xsl:attribute-set>
+
+    <xsl:attribute-set name="layout.master.set.body" use-attribute-sets="layout.master.set.base">
+        <xsl:attribute name="master-name">body</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="layout.master.set.body.region-body">
+        <xsl:attribute name="margin-top">0.7in</xsl:attribute>
+        <xsl:attribute name="margin-bottom">0.8in</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="layout.master.set.body.region-before">
+        <xsl:attribute name="extent">0.35in</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="layout.master.set.body.region-after">
+        <xsl:attribute name="extent">0.125in</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- Style 'primitives' from which all others are descended -->
+    <xsl:attribute-set name="base.body.style">
+        <xsl:attribute name="font-family">Garamond,serif</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="base.heading.style">
+        <xsl:attribute name="font-family">Helvetica,sans-serif</xsl:attribute>
+        <xsl:attribute name="color">#000000</xsl:attribute>
+        <xsl:attribute name="keep-with-next">always</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="base.pre.style">
+        <xsl:attribute name="font-family">monospace</xsl:attribute>
+        <xsl:attribute name="linefeed-treatment">preserve</xsl:attribute>
+        <xsl:attribute name="wrap-option">wrap</xsl:attribute>
+        <xsl:attribute name="keep-together">always</xsl:attribute>
+        <xsl:attribute name="white-space-collapse">false</xsl:attribute>
+    </xsl:attribute-set>
+
+    <xsl:attribute-set name="italic">
+        <xsl:attribute name="font-style">italic</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="bold">
+        <xsl:attribute name="font-weight">bold</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="monospace">
+        <xsl:attribute name="font-family">monospace</xsl:attribute>
+        <xsl:attribute name="font-size">10pt</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- Standard body and heading styles -->
+    <xsl:attribute-set name="body.text" use-attribute-sets="base.body.style">
+        <xsl:attribute name="font-size">11pt</xsl:attribute>
+        <xsl:attribute name="line-height">12pt</xsl:attribute>
+        <xsl:attribute name="white-space-collapse">true</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="body.pre" use-attribute-sets="base.pre.style">
+        <xsl:attribute name="font-size">10pt</xsl:attribute>
+
+    </xsl:attribute-set>
+    <xsl:attribute-set name="code.indent" use-attribute-sets="body.pre">
+        <xsl:attribute name="start-indent">inherited-property-value(start-indent) + 1em</xsl:attribute>
+        <xsl:attribute name="end-indent">inherited-property-value(end-indent) + em</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="body.source" use-attribute-sets="body.pre">
+        <xsl:attribute name="color">black</xsl:attribute>
+        <xsl:attribute name="border-style">solid</xsl:attribute>
+        <xsl:attribute name="border-width">0.5pt</xsl:attribute>
+        <xsl:attribute name="border-color">#454545</xsl:attribute>
+        <xsl:attribute name="padding-before">0.25em</xsl:attribute>
+        <xsl:attribute name="padding-after">0.25em</xsl:attribute>
+        <xsl:attribute name="padding-start">0.25em</xsl:attribute>
+        <xsl:attribute name="padding-end">0.25em</xsl:attribute>
+        <xsl:attribute name="start-indent">inherited-property-value(start-indent) + 2.5em</xsl:attribute>
+        <xsl:attribute name="end-indent">inherited-property-value(end-indent) + 0em</xsl:attribute>
+        <xsl:attribute name="space-before">0.75em</xsl:attribute>
+        <xsl:attribute name="space-after">1em</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="body.rule">
+        <xsl:attribute name="leader-length.optimum">100%</xsl:attribute>
+        <xsl:attribute name="leader-pattern">rule</xsl:attribute>
+        <xsl:attribute name="rule-thickness">0.5pt</xsl:attribute>
+        <xsl:attribute name="color">black</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="body.strong">
+        <xsl:attribute name="font-family">Helvetica,sans-serif</xsl:attribute>
+        <xsl:attribute name="font-size">9.0pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="body.title" use-attribute-sets="base.heading.style">
+        <xsl:attribute name="font-size">16pt</xsl:attribute>
+        <xsl:attribute name="font-weight">bold</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="body.h1" use-attribute-sets="base.heading.style">
+        <xsl:attribute name="font-size">12pt</xsl:attribute>
+        <xsl:attribute name="font-weight">bold</xsl:attribute>
+        <xsl:attribute name="space-before">18pt</xsl:attribute>
+        <xsl:attribute name="space-after">6pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="body.h2" use-attribute-sets="base.heading.style">
+        <xsl:attribute name="font-size">9.5pt</xsl:attribute>
+        <xsl:attribute name="font-weight">bold</xsl:attribute>
+        <xsl:attribute name="space-before">18pt</xsl:attribute>
+        <xsl:attribute name="space-after">5pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="body.h3" use-attribute-sets="base.heading.style">
+        <xsl:attribute name="font-size">9.5pt</xsl:attribute>
+        <xsl:attribute name="space-before">15pt</xsl:attribute>
+        <xsl:attribute name="space-after">3pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="body.h4" use-attribute-sets="base.heading.style">
+        <xsl:attribute name="font-size">9.5pt</xsl:attribute>
+        <xsl:attribute name="space-before">9pt</xsl:attribute>
+        <xsl:attribute name="space-after">3pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="body.h5" use-attribute-sets="base.heading.style">
+        <xsl:attribute name="font-size">9.5pt</xsl:attribute>
+        <xsl:attribute name="font-style">italic</xsl:attribute>
+        <xsl:attribute name="space-after">3pt</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- Chapter heading styles -->
+    <xsl:attribute-set name="chapter.title" use-attribute-sets="body.title">
+        <xsl:attribute name="line-height">10pt</xsl:attribute>
+        <xsl:attribute name="space-after">6pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="chapter.rule">
+        <xsl:attribute name="leader-length.optimum">100%</xsl:attribute>
+        <xsl:attribute name="leader-pattern">dots</xsl:attribute>
+        <xsl:attribute name="rule-thickness">1pt</xsl:attribute>
+        <xsl:attribute name="color">#454545</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- Outdented numbers -->
+    <xsl:attribute-set name="outdented.number.style" use-attribute-sets="base.heading.style">
+        <xsl:attribute name="font-size">9.5pt</xsl:attribute>
+        <!--<xsl:attribute name="color">#454545</xsl:attribute>-->
+        <xsl:attribute name="line-height">10pt</xsl:attribute>
+        <xsl:attribute name="text-align">right</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- Page header/footer styles -->
+    <xsl:attribute-set name="footer.style">
+        <xsl:attribute name="letter-spacing">2pt</xsl:attribute>
+        <xsl:attribute name="font-family">Helvetica,sans-serif</xsl:attribute>
+        <xsl:attribute name="font-size">6pt</xsl:attribute>
+        <xsl:attribute name="color">#454545</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="header.style">
+        <xsl:attribute name="letter-spacing">2pt</xsl:attribute>
+        <xsl:attribute name="font-family">Helvetica,sans-serif</xsl:attribute>
+        <xsl:attribute name="font-size">6pt</xsl:attribute>
+        <xsl:attribute name="color">#454545</xsl:attribute>
+        <xsl:attribute name="text-align">left</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="page.number">
+        <xsl:attribute name="font-family">Helvetica,sans-serif</xsl:attribute>
+        <xsl:attribute name="font-size">9.5pt</xsl:attribute>
+        <xsl:attribute name="text-align">right</xsl:attribute>
+        <xsl:attribute name="color">black</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- Style for hyperlinks -->
+    <xsl:attribute-set name="href.internal">
+        <xsl:attribute name="color">black</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="href.external">
+        <xsl:attribute name="color">black</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- 'Normal' line-spacing styles for paragraph and pre elements -->
+    <xsl:attribute-set name="normal.paragraph" use-attribute-sets="body.text">
+        <xsl:attribute name="space-before">3pt</xsl:attribute>
+        <xsl:attribute name="space-after">6pt</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- Styles for lists, list items, definition lists, etc.  -->
+    <xsl:attribute-set name="list">
+        <xsl:attribute name="start-indent">inherited-property-value(start-indent)</xsl:attribute>
+        <xsl:attribute name="space-before.optimum">10pt</xsl:attribute>
+        <xsl:attribute name="provisional-distance-between-starts">1em</xsl:attribute>
+        <xsl:attribute name="provisional-label-separation">1em</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="list.item">
+        <xsl:attribute name="start-indent">inherited-property-value(start-indent) + 1em</xsl:attribute>
+        <xsl:attribute name="space-before">0.15em</xsl:attribute>
+        <xsl:attribute name="space-after">0.25em</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="dl" use-attribute-sets="body.text">
+        <xsl:attribute name="start-indent">1em</xsl:attribute>
+        <xsl:attribute name="end-indent">1em</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="dt" use-attribute-sets="base.body.style">
+        <xsl:attribute name="start-indent">1em</xsl:attribute>
+        <xsl:attribute name="end-indent">1em</xsl:attribute>
+        <xsl:attribute name="font-weight">bold</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="dd" use-attribute-sets="base.body.style">
+        <xsl:attribute name="start-indent">inherited-property-value(start-indent) + 1em</xsl:attribute>
+        <xsl:attribute name="end-indent">inherited-property-value(end-indent) + 1em</xsl:attribute>
+        <xsl:attribute name="space-before">0.6em</xsl:attribute>
+        <xsl:attribute name="space-after">0.6em</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- Error style -->
+    <xsl:attribute-set name="error.block" use-attribute-sets="base.block">
+        <xsl:attribute name="font-size">8pt</xsl:attribute>
+        <xsl:attribute name="font-weight">bold</xsl:attribute>
+        <xsl:attribute name="color">red</xsl:attribute>
+        <xsl:attribute name="border-style">solid</xsl:attribute>
+        <xsl:attribute name="border-width">0.5pt</xsl:attribute>
+        <xsl:attribute name="border-color">red</xsl:attribute>
+        <xsl:attribute name="padding">0.75em</xsl:attribute>
+        <xsl:attribute name="start-indent">inherited-property-value(start-indent) + 2.5em</xsl:attribute>
+        <xsl:attribute name="end-indent">inherited-property-value(end-indent) + 3em</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- cover styles -->
+    <xsl:attribute-set name="cover.title">
+        <xsl:attribute name="font-family">Helvetica,sans-serif</xsl:attribute>
+        <xsl:attribute name="color">#000000</xsl:attribute>
+        <xsl:attribute name="keep-with-next">always</xsl:attribute>
+        <xsl:attribute name="font-size">16pt</xsl:attribute>
+        <xsl:attribute name="font-weight">bold</xsl:attribute>
+        <xsl:attribute name="text-align">left</xsl:attribute>
+        <xsl:attribute name="display-align">center</xsl:attribute>
+        <xsl:attribute name="space-after">0.5in</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="cover.subtitle">
+        <xsl:attribute name="font-family">Helvetica,sans-serif</xsl:attribute>
+        <xsl:attribute name="color">#000000</xsl:attribute>
+        <xsl:attribute name="keep-with-next">always</xsl:attribute>
+        <xsl:attribute name="font-size">12pt</xsl:attribute>
+        <xsl:attribute name="font-weight">bold</xsl:attribute>
+        <xsl:attribute name="text-align">left</xsl:attribute>
+        <xsl:attribute name="display-align">center</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="cover.border.left" use-attribute-sets="base.pre.style">
+        <xsl:attribute name="padding-start">0.2in</xsl:attribute>
+        <xsl:attribute name="border-left-style">dotted</xsl:attribute>
+        <xsl:attribute name="border-left-width">0.1pt</xsl:attribute>
+        <xsl:attribute name="border-left-color">#000000</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="cover.border.left.bottom" use-attribute-sets="cover.border.left">
+        <xsl:attribute name="padding-after">0.2in</xsl:attribute>
+        <xsl:attribute name="border-bottom-style">dotted</xsl:attribute>
+        <xsl:attribute name="border-bottom-width">0.1pt</xsl:attribute>
+        <xsl:attribute name="border-bottom-color">#000000</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- Header styles (ie title, author, date in single document mode) -->
+    <xsl:attribute-set name="doc.header.title" use-attribute-sets="base.body.style">
+        <xsl:attribute name="text-align">center</xsl:attribute>
+        <xsl:attribute name="font-size">16pt</xsl:attribute>
+        <xsl:attribute name="space-before.optimum">30pt</xsl:attribute>
+        <xsl:attribute name="space-after.optimum">14pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="doc.header.author" use-attribute-sets="base.body.style">
+        <xsl:attribute name="text-align">center</xsl:attribute>
+        <xsl:attribute name="font-size">12pt</xsl:attribute>
+        <xsl:attribute name="space-before.optimum">20pt</xsl:attribute>
+        <xsl:attribute name="space-after.optimum">14pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="doc.header.date" use-attribute-sets="base.body.style">
+        <xsl:attribute name="text-align">center</xsl:attribute>
+        <xsl:attribute name="font-size">12pt</xsl:attribute>
+        <xsl:attribute name="space-before.optimum">20pt</xsl:attribute>
+        <xsl:attribute name="space-after.optimum">30pt</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- Figure styles -->
+    <xsl:attribute-set name="figure.display">
+        <xsl:attribute name="display-align">center</xsl:attribute>
+        <xsl:attribute name="text-align">center</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="figure.graphics">
+        <xsl:attribute name="height">100%</xsl:attribute>
+        <xsl:attribute name="width">100%</xsl:attribute>
+        <xsl:attribute name="content-height">scale-down-to-fit</xsl:attribute>
+        <xsl:attribute name="content-width">scale-down-to-fit</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="figure.caption" use-attribute-sets="base.body.style">
+        <xsl:attribute name="keep-with-previous">always</xsl:attribute>
+        <xsl:attribute name="text-align">center</xsl:attribute>
+        <xsl:attribute name="font-size">10pt</xsl:attribute>
+        <xsl:attribute name="font-style">italic</xsl:attribute>
+        <xsl:attribute name="space-before.optimum">20pt</xsl:attribute>
+        <xsl:attribute name="space-after.optimum">30pt</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- Tables styles -->
+    <xsl:attribute-set name="table.layout">
+        <xsl:attribute name="table-omit-footer-at-break">false</xsl:attribute>
+        <!-- note that table-layout="auto" is not supported by FOP 0.93 -->
+        <xsl:attribute name="table-layout">fixed</xsl:attribute>
+        <xsl:attribute name="width">100%</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.heading.rule">
+        <xsl:attribute name="leader-length.optimum">100%</xsl:attribute>
+        <xsl:attribute name="leader-pattern">rule</xsl:attribute>
+        <xsl:attribute name="rule-thickness">0.5pt</xsl:attribute>
+        <xsl:attribute name="color">black</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="base.cell">
+        <xsl:attribute name="padding-start">2.5pt</xsl:attribute>
+        <xsl:attribute name="padding-end">5pt</xsl:attribute>
+        <!-- http://xmlgraphics.apache.org/fop/faq.html#keep-together -->
+        <xsl:attribute name="keep-together.within-column">always</xsl:attribute>
+        <xsl:attribute name="inline-progression-dimension">auto</xsl:attribute>
+        <xsl:attribute name="overflow">visible</xsl:attribute>
+        <xsl:attribute name="hyphenate">true</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="base.block">
+        <xsl:attribute name="font-family">Helvetica,sans-serif</xsl:attribute>
+        <xsl:attribute name="line-height">1.2em</xsl:attribute>
+
+        <xsl:attribute name="wrap-option">wrap</xsl:attribute>
+        <xsl:attribute name="inline-progression-dimension">auto</xsl:attribute>
+        <xsl:attribute name="overflow">visible</xsl:attribute>
+        <xsl:attribute name="hyphenate">true</xsl:attribute>
+    </xsl:attribute-set>
+
+    <xsl:attribute-set name="table.padding">
+        <xsl:attribute name="padding-before">9pt</xsl:attribute>
+        <xsl:attribute name="padding-after">12pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.title.row">
+        <xsl:attribute name="keep-together">always</xsl:attribute>
+        <xsl:attribute name="keep-with-next">always</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.title.cell" use-attribute-sets="base.cell">
+        <xsl:attribute name="border-after-style">solid</xsl:attribute>
+        <xsl:attribute name="border-after-width">0.5pt</xsl:attribute>
+        <xsl:attribute name="border-after-color">black</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.title.block" use-attribute-sets="base.block">
+        <xsl:attribute name="font-size">11pt</xsl:attribute>
+        <xsl:attribute name="font-weight">bold</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.footer.cell" use-attribute-sets="base.cell">
+        <xsl:attribute name="padding-before">5pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.footer.block" use-attribute-sets="base.block">
+        <xsl:attribute name="font-size">9pt</xsl:attribute>
+        <xsl:attribute name="font-style">italic</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.heading.cell" use-attribute-sets="base.cell">
+        <xsl:attribute name="padding-before">7pt</xsl:attribute>
+        <xsl:attribute name="display-align">after</xsl:attribute>
+        <xsl:attribute name="background-color">#bbbbbb</xsl:attribute>
+        <xsl:attribute name="color">white</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.heading.block" use-attribute-sets="base.block">
+        <xsl:attribute name="font-size">10pt</xsl:attribute>
+        <xsl:attribute name="font-weight">bold</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.heading.rule">
+        <xsl:attribute name="leader-length.optimum">100%</xsl:attribute>
+        <xsl:attribute name="leader-pattern">rule</xsl:attribute>
+        <xsl:attribute name="rule-thickness">0.5pt</xsl:attribute>
+        <xsl:attribute name="color">black</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.number.cell" use-attribute-sets="base.cell">
+        <xsl:attribute name="padding-before">6pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.number.block" use-attribute-sets="base.block">
+        <xsl:attribute name="font-size">9pt</xsl:attribute>
+        <xsl:attribute name="font-style">italic</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.subheading.row">
+        <xsl:attribute name="keep-together">always</xsl:attribute>
+        <xsl:attribute name="keep-with-next">always</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.subheading.cell" use-attribute-sets="base.cell">
+        <xsl:attribute name="padding-after">1pt</xsl:attribute>
+        <xsl:attribute name="background-color">#D3D3D3</xsl:attribute>
+        <xsl:attribute name="border-before-style">solid</xsl:attribute>
+        <xsl:attribute name="border-before-width">2.5pt</xsl:attribute>
+        <xsl:attribute name="border-before-color">#D3D3D3</xsl:attribute>
+        <xsl:attribute name="display-align">after</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.subheading.block" use-attribute-sets="base.block">
+        <xsl:attribute name="font-size">9pt</xsl:attribute>
+        <xsl:attribute name="font-weight">bold</xsl:attribute>
+        <xsl:attribute name="vertical-align">bottom</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.body.row">
+        <xsl:attribute name="keep-together">auto</xsl:attribute>
+        <xsl:attribute name="keep-with-next">auto</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.body.norule">
+        <xsl:attribute name="leader-length.optimum">100%</xsl:attribute>
+        <xsl:attribute name="leader-pattern">rule</xsl:attribute>
+        <xsl:attribute name="rule-thickness">1pt</xsl:attribute>
+        <xsl:attribute name="color">white</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.body.rule">
+        <xsl:attribute name="leader-length.optimum">100%</xsl:attribute>
+        <xsl:attribute name="leader-pattern">dots</xsl:attribute>
+        <xsl:attribute name="rule-thickness">0.5pt</xsl:attribute>
+        <xsl:attribute name="color">#A9A9A9</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.body.lastrule">
+        <xsl:attribute name="leader-length.optimum">100%</xsl:attribute>
+        <xsl:attribute name="leader-pattern">rule</xsl:attribute>
+        <xsl:attribute name="rule-thickness">0.5pt</xsl:attribute>
+        <xsl:attribute name="color">black</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.body.cell.grid">
+        <xsl:attribute name="border-style">solid</xsl:attribute>
+        <xsl:attribute name="border-width">0.2mm</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.body.cell" use-attribute-sets="base.cell">
+        <xsl:attribute name="padding-before">4pt</xsl:attribute>
+        <xsl:attribute name="padding-after">1.5pt</xsl:attribute>
+        <xsl:attribute name="background-color">#eeeeee</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.body.block" use-attribute-sets="base.block">
+        <xsl:attribute name="font-size">9pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="table.pre" use-attribute-sets="base.pre.style">
+        <xsl:attribute name="font-size">9pt</xsl:attribute>
+    </xsl:attribute-set>
+
+    <!-- Table of content styles -->
+    <xsl:attribute-set name="toc.cell">
+        <xsl:attribute name="display-align">after</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="toc.base.style">
+        <xsl:attribute name="font-family">Helvetica,sans-serif</xsl:attribute>
+        <xsl:attribute name="line-height">16pt</xsl:attribute>
+        <!-- <xsl:attribute name="text-align-last">start</xsl:attribute>-->
+        <xsl:attribute name="text-align-last">justify</xsl:attribute>
+        <xsl:attribute name="wrap-option">no-wrap</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="toc.leader.style">
+        <xsl:attribute name="leader-pattern">dots</xsl:attribute>
+        <xsl:attribute name="leader-pattern-width">5pt</xsl:attribute>
+        <xsl:attribute name="color">#454545</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="toc.number.style">
+        <xsl:attribute name="font-size">9.5pt</xsl:attribute>
+        <xsl:attribute name="text-align">end</xsl:attribute>
+        <xsl:attribute name="color">#A9A9A9</xsl:attribute>
+        <xsl:attribute name="line-height">16pt</xsl:attribute>
+        <xsl:attribute name="end-indent">6pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="toc.h1.style" use-attribute-sets="toc.base.style">
+        <xsl:attribute name="font-size">12pt</xsl:attribute>
+        <xsl:attribute name="font-weight">bold</xsl:attribute>
+        <xsl:attribute name="space-before">18pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="toc.h2.style" use-attribute-sets="toc.base.style">
+        <xsl:attribute name="font-size">11pt</xsl:attribute>
+        <xsl:attribute name="space-before">15pt</xsl:attribute>
+        <xsl:attribute name="space-before">3pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="toc.h3.style" use-attribute-sets="toc.base.style">
+        <xsl:attribute name="font-size">10pt</xsl:attribute>
+        <xsl:attribute name="space-before">4pt</xsl:attribute>
+    </xsl:attribute-set>
+    <xsl:attribute-set name="toc.h4.style" use-attribute-sets="toc.base.style">
+        <xsl:attribute name="font-size">9.5pt</xsl:attribute>
+        <xsl:attribute name="space-before">4pt</xsl:attribute>
+    </xsl:attribute-set>
+</xsl:stylesheet>
\ No newline at end of file


[5/9] carbondata-site git commit: Added Functionality of PdfGeneration

Posted by ch...@apache.org.
Added Functionality of PdfGeneration


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

Branch: refs/heads/asf-site
Commit: 069b05da4fbf4b9e6ed9647ce9a4b2846c137507
Parents: 88ecec5
Author: jatin <ja...@knoldus.in>
Authored: Tue May 23 12:14:39 2017 +0530
Committer: jatin <ja...@knoldus.in>
Committed: Tue May 23 12:14:39 2017 +0530

----------------------------------------------------------------------
 carbonscript.sh                                 |   2 +
 pom.xml                                         |  31 +-
 src/main/resources/application.conf             |   3 +-
 src/main/scala/MDFileConverter.scala            |   1 +
 src/site/images/CarbonData_logo.png             | Bin 0 -> 10971 bytes
 .../images/carbon_data_file_structure_new.png   | Bin 0 -> 9477 bytes
 src/site/images/carbon_data_format_new.png      | Bin 0 -> 35510 bytes
 src/site/markdown/configuration-parameters.md   | 149 ------
 src/site/markdown/data-management.md            | 166 ------
 .../markdown/ddl-operation-on-carbondata.md     | 231 --------
 .../markdown/dml-operation-on-carbondata.md     | 451 ----------------
 src/site/markdown/faq.md                        |  77 ---
 .../markdown/file-structure-of-carbondata.md    |  45 --
 src/site/markdown/installation-guide.md         | 200 -------
 src/site/markdown/quick-start-guide.md          | 143 -----
 .../supported-data-types-in-carbondata.md       |  41 --
 src/site/markdown/troubleshooting.md            | 247 ---------
 src/site/markdown/useful-tips-on-carbondata.md  | 180 -------
 src/site/pdf.xml                                |  35 ++
 src/site/projectLogo/ApacheLogo.png             | Bin 0 -> 7241 bytes
 src/site/projectLogo/CarbonDataLogo.png         | Bin 0 -> 10971 bytes
 src/site/resources/pdf-config.xml               | 520 +++++++++++++++++++
 22 files changed, 577 insertions(+), 1945 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/carbonscript.sh
----------------------------------------------------------------------
diff --git a/carbonscript.sh b/carbonscript.sh
index 0ec7143..538922d 100755
--- a/carbonscript.sh
+++ b/carbonscript.sh
@@ -3,3 +3,5 @@ mvn compile
 mvn exec:java -Dexec.mainClass=WebsiteLauncher
 
 mvn install
+
+mvn pdf:pdf
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/pom.xml
----------------------------------------------------------------------
diff --git a/pom.xml b/pom.xml
index 0c1a1cd..a863151 100644
--- a/pom.xml
+++ b/pom.xml
@@ -143,6 +143,13 @@
                     </configuration>
                 </execution>
             </executions>
+            <dependencies>
+                <dependency>
+                    <groupId>org.apache.maven.doxia</groupId>
+                    <artifactId>doxia-module-markdown</artifactId>
+                    <version>1.6</version>
+                </dependency>
+            </dependencies>
         </plugin>
         <plugin>
             <groupId>org.apache.maven.plugins</groupId>
@@ -150,36 +157,36 @@
             <version>3.0.2</version>
             <executions>
                 <execution>
-                    <id>publish-pdf</id>
+                    <id>publish-site</id>
                     <phase>install</phase>
                     <goals>
                         <goal>copy-resources</goal>
                     </goals>
                     <configuration>
-                        <outputDirectory>${basedir}/src/pdf</outputDirectory>
+                        <outputDirectory>${basedir}/content</outputDirectory>
                         <resources>
                             <resource>
-                                <directory>${basedir}/target/pdf</directory>
+                                <directory>${basedir}/target/site-17</directory>
                                 <filtering>false</filtering>
-                                <includes>
-                                    <include>**/*.pdf</include>
-                                </includes>
                             </resource>
                         </resources>
                     </configuration>
                 </execution>
                 <execution>
-                    <id>publish-site</id>
+                    <id>publish-pdf</id>
                     <phase>install</phase>
                     <goals>
                         <goal>copy-resources</goal>
                     </goals>
                     <configuration>
-                        <outputDirectory>${basedir}/content</outputDirectory>
+                        <outputDirectory>${basedir}/src/pdf</outputDirectory>
                         <resources>
                             <resource>
-                                <directory>${basedir}/target/site-17</directory>
+                                <directory>${basedir}/target/pdf</directory>
                                 <filtering>false</filtering>
+                                <includes>
+                                    <include>**/*.pdf</include>
+                                </includes>
                             </resource>
                         </resources>
                     </configuration>
@@ -214,11 +221,6 @@
       <artifactId>scala-library</artifactId>
       <version>2.12.1</version>
     </dependency>
-      <dependency>
-          <groupId>org.apache.maven.doxia</groupId>
-          <artifactId>doxia-module-markdown</artifactId>
-          <version>1.6</version>
-      </dependency>
     <!-- https://mvnrepository.com/artifact/org.apache.httpcomponents/httpclient -->
     <dependency>
       <groupId>org.apache.httpcomponents</groupId>
@@ -266,5 +268,6 @@
       <version>3.0</version>
     </dependency>
   </dependencies>
+
 </project>
 

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/src/main/resources/application.conf b/src/main/resources/application.conf
index b0b9ff6..f7f5a09 100644
--- a/src/main/resources/application.conf
+++ b/src/main/resources/application.conf
@@ -16,4 +16,5 @@ 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/"
-mdLink="https://api.github.com/markdown/raw"
\ No newline at end of file
+mdLink="https://api.github.com/markdown/raw"
+mdFileLocation="src/site/markdown/"
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/main/scala/MDFileConverter.scala
----------------------------------------------------------------------
diff --git a/src/main/scala/MDFileConverter.scala b/src/main/scala/MDFileConverter.scala
index b4649d2..4fab013 100644
--- a/src/main/scala/MDFileConverter.scala
+++ b/src/main/scala/MDFileConverter.scala
@@ -27,6 +27,7 @@ class MDFileConverter @Inject()(fileService: FileService, confService: ConfServi
         case Some(data: String) => val fileData = fileReadObject.convertMdExtensions(data)
           logger.info(s"Begin writing [ $file outputFileExtension ] at $location")
           val statusHtmlFile = fileService.writeToFile(location + file + outputFileExtension, headerContent + fileData + footerContent)
+          val statusMdFile = fileService.writeToFile(confService.readString("mdFileLocation") + file + inputFileExtension, fileURLContent)
           if (statusHtmlFile) {
             logger.info(s"Successfully written [ $file $outputFileExtension ] at $location")
             "Success"

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/site/images/CarbonData_logo.png
----------------------------------------------------------------------
diff --git a/src/site/images/CarbonData_logo.png b/src/site/images/CarbonData_logo.png
new file mode 100644
index 0000000..bc09b23
Binary files /dev/null and b/src/site/images/CarbonData_logo.png differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/site/images/carbon_data_file_structure_new.png
----------------------------------------------------------------------
diff --git a/src/site/images/carbon_data_file_structure_new.png b/src/site/images/carbon_data_file_structure_new.png
new file mode 100644
index 0000000..1c6f22b
Binary files /dev/null and b/src/site/images/carbon_data_file_structure_new.png differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/site/images/carbon_data_format_new.png
----------------------------------------------------------------------
diff --git a/src/site/images/carbon_data_format_new.png b/src/site/images/carbon_data_format_new.png
new file mode 100644
index 0000000..f0fc553
Binary files /dev/null and b/src/site/images/carbon_data_format_new.png differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/site/markdown/configuration-parameters.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/configuration-parameters.md b/src/site/markdown/configuration-parameters.md
deleted file mode 100644
index 774734a..0000000
--- a/src/site/markdown/configuration-parameters.md
+++ /dev/null
@@ -1,149 +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.
--->
-
-# Configuring CarbonData
- This tutorial guides you through the advanced configurations of CarbonData :
- 
- * [System Configuration](#system-configuration)
- * [Performance Configuration](#performance-configuration)
- * [Miscellaneous Configuration](#miscellaneous-configuration)
- * [Spark Configuration](#spark-configuration)
- 
- 
-##  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 | /user/hive/warehouse/carbon.store | Location where CarbonData will create the store, and write the data in its own format. NOTE: Store location should be in HDFS. |
-| carbon.ddl.base.hdfs.url | hdfs://hacluster/opt/data | 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 | /opt/Carbon/Spark/badrecords | Path where the bad records are stored. |
-| carbon.kettle.home | $SPARK_HOME/carbonlib/carbonplugins | Configuration for loading the data with kettle. |
-| carbon.data.file.version | 2 | 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.|                    
-
-##  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.sort.file.buffer.size | 20 | File read buffer size used during sorting. This value is expressed in MB. | Min=1 and Max=100 |
-| carbon.graph.rowset.size | 100000 | Rowset size exchanged between data load graph steps. | Min=500 and Max=1000000 |
-| carbon.number.of.cores.while.loading | 6 | Number of cores to be used while loading data. |  |
-| carbon.sort.size | 500000 | Record count to sort and write intermediate files to temp. |  |
-| carbon.enableXXHash | true | Algorithm for hashmap for hashkey calculation. |  |
-| carbon.number.of.cores.block.sort | 7 | Number of cores to use for block sort while loading data. |  |
-| 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.update.persist.enable | true | Enabling this parameter considers persistent data. Enabling this will reduce the execution time of UPDATE operation. |  |
-
-
-
-* **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. |
-
-  
-
-* **Query Configuration**
-  
-| Parameter | Default Value | Description | Range |
-|--------------------------------------|---------------|---------------------------------------------------|---------------------------|
-| carbon.number.of.cores | 4 | Number of cores to be used while querying. |  |
-| carbon.inmemory.record.size | 120000 | Number of records to be in memory while querying. | Min=100000 and Max=240000 |
-| carbon.enable.quick.filter | false | Improves the performance of filter query. |  |
-| no.of.cores.to.load.blocks.in.driver | 10 | Number of core to load the blocks in driver. |  |
-
-
-##   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 | 10485760 | File write buffer size used during sorting. |
-| 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.sort.intermediate.files.limit | 20 | Minimum number of intermediate files after which merged sort can be started. |
-| 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. |
-| high.cardinality.value | 100000 | To identify and apply compression for non-high cardinality columns. |
-| carbon.merge.sort.reader.thread | 3 | Maximum no of threads used for reading intermediate files for final merging. |
-| carbon.load.metadata.lock.retries | 3 | Maximum number of retries to get the metadata lock for loading data to table. |
-| carbon.load.metadata.lock.retry.timeout.sec | 5 | Interval between the retries to get the lock. |
-| carbon.tempstore.location | /opt/Carbon/TempStoreLoc | Temporary store location. By default it takes System.getProperty("java.io.tmpdir"). |
-| carbon.load.log.counter | 500000 | Data loading records count logger. |   
-
-
-* **Compaction 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. |
-
- 
-* **Query Configuration**
-
-| Parameter | Default Value | Description |
-|--------------------------|---------------|-----------------------------------------------------------------------------------------------|
-| 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. | 
-  
-* **Global Dictionary Configurations**
-  
-| Parameter | Default Value | Description |
-|---------------------------------------|---------------------|-------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
-| high.cardinality.identify.enable | true | If the parameter is true, the high cardinality columns of the dictionary code are automatically recognized and these columns will not be used as global dictionary encoding. If the parameter is false, all dictionary encoding columns are used as dictionary encoding. The high cardinality column must meet the following requirements: value of cardinality > configured value of high.cardinalityEqually, the value of cardinality is higher than the threshold.value of cardinality/ row number x 100 > configured value of high.cardinality.row.count.percentageEqually, the ratio of the cardinality value to data row number is higher than the configured percentage. |
-| high.cardinality.threshold | 1000000 | high.cardinality.threshold | 1000000 | It is a threshold to identify high cardinality of the columns.If the value of columns' cardinality > the configured value, then the columns are excluded from dictionary encoding. |
-| high.cardinality.row.count.percentage | 80 | Percentage to identify whether column cardinality is more than configured percent of total row count.Configuration value formula:Value of cardinality/ row number x 100 > configured value of high.cardinality.row.count.percentageThe value of the parameter must be larger than 0. |
-| carbon.cutOffTimestamp | 1970-01-01 05:30:00 | 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". NOTE: The CarbonData supports data store up to 68 years from the cut-off time defined. For example, if the cut-off time is 1970-01-01 05:30:00, then the data can be stored up to 2038-01-01 05:30:00. |
-| 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>
- 
-| 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. |
-| spark.sql.bigdata.register.analyseRule | org.apache.spark.sql.hive.acl.CarbonAccessControlRules | CarbonAccessControlRules need to be set for enabling Access Control. |
-   
- 
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/site/markdown/data-management.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/data-management.md b/src/site/markdown/data-management.md
deleted file mode 100644
index 2663aff..0000000
--- a/src/site/markdown/data-management.md
+++ /dev/null
@@ -1,166 +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
-This tutorial is going to introduce you to the conceptual details of data management like:
-
-* [Loading Data](#loading-data)
-* [Deleting Data](#deleting-data)
-* [Compacting Data](#compacting-data)
-* [Updating Data](#updating-data)
-
-## Loading Data
-
-* **Scenario**
-
-   After creating a table, you can load data to the table using the [LOAD DATA](dml-operation-on-carbondata.md) 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.
-
-* **Procedure**
-  
-   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.
-   
-   
-| Status | Description |
-|-----------------|------------------------------------------------------------------------------------------------------------|
-| Success | All the data is loaded into table and no bad records found. |
-| Partial Success | Data is loaded into table and bad records are found. Bad records are stored at carbon.badrecords.location. |
-   
-   In case of failure, the error will be logged in error log. Details of loads can be seen with [SHOW SEGMENTS](dml-operation-on-carbondata.md) command. The show segment command output consists of :
-   
-   - SegmentSequenceID
-   - START_TIME OF LOAD
-   - END_TIME OF LOAD 
-   - LOAD STATUS
- 
-   The latest load will be displayed first in the output.
-   
-   Refer to [DML operations on CarbonData](dml-operation-on-carbondata.md) for load commands.
-   
-   
-## Deleting Data  
-
-* **Scenario**
-   
-   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.
-   
-* **Procedure** 
-
-   The loaded data can be deleted in the following ways:
-   
-   * Delete by Segment ID
-      
-      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.
-      
-| SegmentSequenceId | Status            | Load Start Time      | Load End Time        |
-|-------------------|-------------------|----------------------|----------------------|
-| 0                 | Success           | 2015-11-19 19:14:... | 2015-11-19 19:14:... |
-| 1                 | Marked for Update | 2015-11-19 19:54:... | 2015-11-19 20:08:... |
-| 2                 | Marked for Delete | 2015-11-19 20:25:... | 2015-11-19 20:49:... |
-
-   * Delete by Date Field
-   
-      If the table contains date field, you can delete the data based on a specific date.
-
-   * Delete by Record
-
-      To delete records from CarbonData table based on some filter Condition(s).
-      
-      For delete commands refer to [DML operations on CarbonData](dml-operation-on-carbondata.md).
-      
-   * **NOTE**:
-    
-     - 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.
-     
-     - 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.
-     
-     - If the user wants to force delete the segment physically then he can use CLEAN FILES Command.
-        
-Example :
-          
-```
-CLEAN FILES FOR TABLE table1
-```
-
- This DML will physically delete the segment which are "Marked for delete" immediately.
-
-## Compacting Data
-      
-* **Scenario**
-  
-  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.  
-      
-* **Procedure**
-
-  There are two types of compaction Minor and Major compaction.
-  
-  - **Minor Compaction**
-    
-     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.
-     
-     - Level 1: Merging of the segments which are not yet compacted.
-     - Level 2: Merging of the compacted segments again to form a bigger segment. 
-  - **Major Compaction**
-     
-     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. 
-      
-   There are number of parameters related to Compaction that can be set in carbon.properties file 
-   
-| Parameter | Default | Application | Description | Valid Values |
-|-----------------------------------------|---------|-------------|--------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|--------------|
-| carbon.compaction.level.threshold | 4, 3 | Minor | 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. | NA |
-| carbon.major.compaction.size | 1024 MB | Major | Major compaction size can be configured using this parameter. Sum of the segments which is below this threshold will be merged. | NA |
-| carbon.numberof.preserve.segments | 0 | Minor/Major | 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. | 0-100 |
-| carbon.allowed.compaction.days | 0 | Minor/Major | 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. | 0-100 |
-| carbon.number.of.cores.while.compacting | 2 | Minor/Major | Number of cores which is used to write data during compaction. | 0-100 |   
-  
-   For compaction commands refer to [DDL operations on CarbonData](ddl-operation-on-carbondata.md)
-
-## Updating Data
-
-* **Scenario**
-
-    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.
-
-* **Procedure**
-
-    To update we need to specify the column expression with an optional filter condition(s).
-
-    For update commands refer to [DML operations on CarbonData](dml-operation-on-carbondata.md).
-
-
-    
-
-
-
-
- 
- 

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/site/markdown/ddl-operation-on-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/ddl-operation-on-carbondata.md b/src/site/markdown/ddl-operation-on-carbondata.md
deleted file mode 100644
index de4999e..0000000
--- a/src/site/markdown/ddl-operation-on-carbondata.md
+++ /dev/null
@@ -1,231 +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.
--->
-
-# DDL Operations on CarbonData
-This tutorial guides you through the data definition language support provided by CarbonData.
-
-## Overview 
-The following DDL operations are supported in CarbonData :
-
-* [CREATE TABLE](#create-table)
-* [SHOW TABLE](#show-table)
-* [DROP TABLE](#drop-table)
-* [COMPACTION](#compaction)
-* [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.
-```
-   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
-```
-
-### Parameter Description
-
-| Parameter | Description | Optional |
-|---------------|-----------------------------------------------------------------------------------------------------------------------------------------------|----------|
-| db_name | Name of the database. Database name should consist of alphanumeric characters and underscore(_) special character. | Yes |
-| field_list | Comma separated List of fields with data type. The field names should consist of alphanumeric characters and underscore(_) special character. | No |
-| table_name | The name of the table in Database. Table Name should consist of alphanumeric characters and underscore(_) special character. | No |
-| STORED BY | "org.apache.carbondata.format", identifies and creates a CarbonData table. | No |
-| TBLPROPERTIES | List of CarbonData table properties. |  |
-
-### Usage Guidelines
-
-   Following are the guidelines for using table properties.
-
-   - **Dictionary Encoding Configuration**
-
-       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.
-```
-       TBLPROPERTIES ("DICTIONARY_EXCLUDE"="column1, column2")
-       TBLPROPERTIES ("DICTIONARY_INCLUDE"="column1, column2")
-```
-
-   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.
-
-   - **Row/Column Format Configuration**
-
-       Column groups with more than one column are stored in row format, instead of columnar format. By default, each column is a separate column group.
-```
-TBLPROPERTIES ("COLUMN_GROUPS"="(column1, column3),
-(Column4,Column5,Column6)")
-```
-
-   - **Table Block Size Configuration**
-
-     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.
-```
-       TBLPROPERTIES ("TABLE_BLOCKSIZE"="512 MB")
-```
-
-  Here 512 MB means the block size of this table is 512 MB, you can also set it as 512M or 512.
-
-   - **Inverted Index Configuration**
-
-      Inverted index is very useful to improve compression ratio and query speed, especially for those low-cardinality columns who are in reward position.
-      By default inverted index is enabled. The user can disable the inverted index creation for some columns.
-```
-       TBLPROPERTIES ("NO_INVERTED_INDEX"="column1, column3")
-```
-
-  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.
-
-   NOTE:
-
-   - By default all columns other than numeric datatype are treated as dimensions and all columns of numeric datatype are treated as measures.
-
-   - 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.
-
-### 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 ('COLUMN_GROUPS'='(productNumber,productName)',
-                     'DICTIONARY_EXCLUDE'='storeCity',
-                     'DICTIONARY_INCLUDE'='productNumber',
-                     'NO_INVERTED_INDEX'='productBatch')
-```
-
-## 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];
-```
-
-### Parameter Description
-| Parameter  | Description                                                                               | Optional |
-|------------|-------------------------------------------------------------------------------------------|----------|
-| IN db_Name | Name of the database. Required only if tables of this specific database are to be listed. | Yes      |
-
-### Example:
-```
-  SHOW TABLES IN ProductSchema;
-```
-
-## DROP TABLE
-
- This command is used to delete an existing table.
-```
-  DROP TABLE [IF EXISTS] [db_name.]table_name;
-```
-
-### Parameter Description
-| Parameter | Description | Optional |
-|-----------|-------------| -------- |
-| db_Name | Name of the database. If not specified, current database will be selected. | YES |
-| table_name | Name of the table to be deleted. | NO |
-
-### Example:
-```
-  DROP TABLE IF EXISTS productSchema.productSalesTable;
-```
-
-## COMPACTION
-
-This command merges the specified number of segments into one segment. This enhances the query performance of the table.
-```
-  ALTER TABLE [db_name.]table_name COMPACT 'MINOR/MAJOR';
-```
-
-  To get details about Compaction refer to [Data Management](data-management.md)
-
-### Parameter Description
-
-| Parameter | Description | Optional |
-| ------------- | -----| ----------- |
-| db_name | Database name, if it is not specified then it uses current database. | YES |
-| table_name | The name of the table in provided database.| NO |
-
-### Syntax
-
-- **Minor Compaction**
-```
-ALTER TABLE table_name COMPACT 'MINOR';
-```
-- **Major Compaction**
-```
-ALTER TABLE table_name COMPACT 'MAJOR';
-```
-
-## 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, a user needs to specify the
-columns to be used for bucketing and the number of buckets. For the selction 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”)
-```
-  
-## Parameter Description
-
-| Parameter 	| Description 	| Optional 	|
-|---------------	|------------------------------------------------------------------------------------------------------------------------------	|----------	|
-| BUCKETNUMBER 	| Specifies the number of Buckets to be created. 	| No 	|
-| BUCKETCOLUMNS 	| Specify the columns to be considered for Bucketing  	| No 	|
-
-## Usage Guidelines
-
-- The feature is supported for Spark 1.6.2 onwards, but the performance optimization is evident from Spark 2.1 onwards.
-
-- Bucketing can not be performed for columns of Complex Data Types.
-
-- Columns in the BUCKETCOLUMN parameter must be only dimension. The BUCKETCOLUMN parameter can not be a measure or a combination of measures and dimensions.
-
-
-## Example :
-
-```
- CREATE TABLE IF NOT EXISTS productSchema.productSalesTable (
-                                productNumber Int,
-                                productName String,
-                                storeCity String,
-                                storeProvince String,
-                                productCategory String,
-                                productBatch String,
-                                saleQuantity Int,
-                                revenue Int)
-   STORED BY 'carbondata'
-   TBLPROPERTIES ('COLUMN_GROUPS'='(productName,productNumber)',
-                  'DICTIONARY_EXCLUDE'='productName',
-                  'DICTIONARY_INCLUDE'='productNumber',
-                  'NO_INVERTED_INDEX'='productBatch',
-                  'BUCKETNUMBER'='4',
-                  'BUCKETCOLUMNS'='productName')
- ```
-

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/site/markdown/dml-operation-on-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/dml-operation-on-carbondata.md b/src/site/markdown/dml-operation-on-carbondata.md
deleted file mode 100644
index 74fa0b0..0000000
--- a/src/site/markdown/dml-operation-on-carbondata.md
+++ /dev/null
@@ -1,451 +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.
--->
-
-# DML Operations on CarbonData
-This tutorial guides you through the data manipulation language support provided by CarbonData.
-
-## Overview 
-The following DML operations are supported in CarbonData :
-
-* [LOAD DATA](#load-data)
-* [INSERT DATA INTO A CARBONDATA TABLE](#insert-data-into-a-carbondata-table)
-* [SHOW SEGMENTS](#show-segments)
-* [DELETE SEGMENT BY ID](#delete-segment-by-id)
-* [DELETE SEGMENT BY DATE](#delete-segment-by-date)
-* [UPDATE CARBONDATA TABLE](#update-carbondata-table)
-* [DELETE RECORDS FROM CARBONDATA TABLE](#delete-records-from-carbondata-table)
-
-## LOAD DATA
-
-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 [Data Management](data-management.md) for more details on LOAD.
-
-### Syntax
-
-```
-LOAD DATA [LOCAL] INPATH 'folder_path' 
-INTO TABLE [db_name.]table_name 
-OPTIONS(property_name=property_value, ...)
-```
-
-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.
-
-NOTE: The path shall be canonical path.
-
-### Parameter Description
-
-| Parameter     | Description                                                          | Optional |
-| ------------- | ---------------------------------------------------------------------| -------- |
-| folder_path   | Path of raw csv data folder or file.                                 | NO       |
-| db_name       | Database name, if it is not specified then it uses the current database. | YES      |
-| table_name    | The name of the table in provided database.                          | NO       |
-| OPTIONS       | Extra options provided to Load                                       | YES      |
- 
-
-### Usage Guidelines
-
-You can use the following options to load data:
-
-- **DELIMITER:** Delimiters can be provided in the load command.
-    
-    ``` 
-    OPTIONS('DELIMITER'=',')
-    ```
-
-- **QUOTECHAR:** Quote Characters can be provided in the load command.
-
-    ```
-    OPTIONS('QUOTECHAR'='"')
-    ```
-
-- **COMMENTCHAR:** Comment Characters can be provided in the load command if user want to comment lines.
-
-    ```
-    OPTIONS('COMMENTCHAR'='#')
-    ```
-
-- **FILEHEADER:** Headers can be provided in the LOAD DATA command if headers are missing in the source files.
-
-    ```
-    OPTIONS('FILEHEADER'='column1,column2') 
-    ```
-
-- **MULTILINE:** CSV with new line character in quotes.
-
-    ```
-    OPTIONS('MULTILINE'='true') 
-    ```
-
-- **ESCAPECHAR:** Escape char can be provided if user want strict validation of escape character on CSV.
-
-    ```
-    OPTIONS('ESCAPECHAR'='\') 
-    ```
-
-- **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:** Date format for specified column.
-
-    ```
-    OPTIONS('DATEFORMAT'='column1:dateFormat1, column2:dateFormat2')
-    ```
-
-    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).
-
-- **USE_KETTLE:** This option is used to specify whether to use kettle for loading data or not. By default kettle is not used for data loading.
-
-    ```
-    OPTIONS('USE_KETTLE'='FALSE')
-    ```
-
-   Note :  It is recommended to set the value for this option as false.
-
-- **SINGLE_PASS:** Single Pass Loading enables single job to finish data loading with dictionary generation on the fly. It enhances performance in the scenarios where the subsequent data loading after initial load involves fewer incremental updates on the dictionary.
-
-   This option specifies whether to use single pass for loading data or not. By default this option is set to FALSE.
-
-    ```
-    OPTIONS('SINGLE_PASS'='TRUE')
-    ```
-
-   Note :
-
-   * If this option is set to TRUE then data loading will take less time.
-
-   * If this option is set to some invalid value other than TRUE or FALSE then it uses the default value.
-### Example:
-
-```
-LOAD DATA local inpath '/opt/rawdata/data.csv' INTO table carbontable
-options('DELIMITER'=',', 'QUOTECHAR'='"','COMMENTCHAR'='#',
-'FILEHEADER'='empno,empname,designation,doj,workgroupcategory,
- workgroupcategoryname,deptno,deptname,projectcode,
- projectjoindate,projectenddate,attendance,utilization,salary',
-'MULTILINE'='true','ESCAPECHAR'='\','COMPLEX_DELIMITER_LEVEL_1'='$',
-'COMPLEX_DELIMITER_LEVEL_2'=':',
-'ALL_DICTIONARY_PATH'='/opt/alldictionary/data.dictionary',
-'USE_KETTLE'='FALSE',
-'SINGLE_PASS'='TRUE'
-)
-```
-
-
-## INSERT DATA INTO A 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.
-
-**NOTE** :  The client node where the INSERT command is executing, must be part of the cluster.
-
-### Syntax
-
-```
-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> } ];
-```
-
-### Parameter Description
-
-| Parameter | Description |
-|--------------|---------------------------------------------------------------------------------|
-| CARBON TABLE | The name of the Carbon table in which you want to perform the insert operation. |
-| sourceTableName | The table from which the records are read and inserted into destination CarbonData table. |
-
-### Usage Guidelines
-The following condition must be met for successful insert operation :
-
-- The source table and the CarbonData table must have the same table schema.
-- The table must be created.
-- Overwrite is not supported for CarbonData table.
-- 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.
-- 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.
-
-To enable data load or update during insert operation, configure the following property to true.
-
-```
-carbon.insert.persist.enable=true
-```
-
-By default the above configuration will be false.
-
-**NOTE**: Enabling this property will reduce the performance.
-
-### 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 INTO table1 SELECT * FROM table2 
-where exists (select * from table3 
-where table2.item1 = table3.item1);
-```
-
-**The Status Success/Failure shall be captured in the driver log.**
-
-## SHOW SEGMENTS
-
-This command is used to get the segments of CarbonData table.
-
-```
-SHOW SEGMENTS FOR TABLE [db_name.]table_name 
-LIMIT number_of_segments;
-```
-
-### Parameter Description
-
-| Parameter          | Description                                                          | Optional |
-| ------------------ | ---------------------------------------------------------------------| ---------|
-| db_name            | Database name, if it is not specified then it uses the current database. | YES      |
-| table_name         | The name of the table in provided database.                          | NO       |
-| number_of_segments | Limit the output to this number.                                     | YES      |
-
-### Example:
-
-```
-SHOW SEGMENTS FOR TABLE CarbonDatabase.CarbonTable LIMIT 4;
-```
-
-## DELETE SEGMENT BY ID
-
-This command is used to delete segment by using the segment ID. Each segment has a unique segment ID associated with it. 
-Using this segment ID, you can remove the segment.
-
-The following command will get the segmentID.
-
-```
-SHOW SEGMENTS FOR Table dbname.tablename 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 SEGMENT segment_sequence_id1, segments_sequence_id2, .... 
-FROM TABLE tableName
-```
-
-### Parameter Description
-| Parameter  | Description                                                          | Optional |
-| -----------| ---------------------------------------------------------------------|----------|
-| segment_id | Segment Id of the load.                                              | NO       |
-| db_name    | Database name, if it is not specified then it uses the current database. | YES      |
-| table_name | The name of the table in provided database.                          | NO       |
-
-### Example:
-
-```
-DELETE SEGMENT 0 FROM TABLE CarbonDatabase.CarbonTable;
-DELETE SEGMENT 0.1,5,8 FROM TABLE CarbonDatabase.CarbonTable;
-```
-  NOTE: Here 0.1 is compacted segment sequence id. 
-
-## 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 [schema_name.]table_name 
-WHERE[DATE_FIELD]BEFORE [DATE_VALUE]
-```
-
-### Parameter Description
-
-| Parameter  | Description                                                                                        | Optional |
-| ---------- | ---------------------------------------------------------------------------------------------------| -------- |
-| DATE_VALUE | Valid segment load start time value. All the segments before this specified date will be deleted. | NO       |
-| db_name    | Database name, if it is not specified then it uses the current database.                               | YES      |
-| table_name | The name of the table in provided database.                                                        | NO       |
-
-### Example:
-
-```
- DELETE SEGMENTS FROM TABLE CarbonDatabase.CarbonTable 
- WHERE STARTTIME BEFORE '2017-06-01 12:05:06';  
-```
-
-## Update CarbonData Table
-This command will allow to update the carbon table based on the column expression and optional filter conditions.
-
-### Syntax
-
-```
- UPDATE <table_name>
- SET (column_name1, column_name2, ... column_name n) =
- (column1_expression , column2_expression . .. column n_expression )
- [ WHERE { <filter_condition> } ];
-```
-
-alternatively the following the command can also be used for updating the CarbonData Table :
-
-```
-UPDATE <table_name>
-SET (column_name1, column_name2,) =
-(select sourceColumn1, sourceColumn2 from sourceTable
-[ WHERE { <filter_condition> } ] )
-[ WHERE { <filter_condition> } ];
-```
-
-### Parameter Description
-
-| Parameter | Description |
-|--------------|---------------------------------------------------------------------------------|
-| table_name | The name of the Carbon table in which you want to perform the update operation. |
-| column_name | The destination columns to be updated. |
-| sourceColumn | The source table column values to be updated in destination table. |
-| sourceTable | The table from which the records are updated into destination Carbon table. |
-
-### Usage Guidelines
-The following conditions must be met for successful updation :
-
-- The update command fails if multiple input rows in source table are matched with single row in destination table.
-- If the source table generates empty records, the update operation will complete successfully without updating the table.
-- 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.
-- In sub-query, if the source table and the target table are same, then the update operation fails.
-- If the sub-query used in UPDATE statement contains aggregate method or group by query, then the UPDATE operation fails.
-
-### Examples
-
- Update is not supported for queries that contain aggregate or group by.
-
-```
- 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);
-```
-
-Here the Update Operation fails as the query contains aggregate function sum(b.profit) and group by clause in the sub-query.
-
-
-```
-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 > 1);
-```
-
-
-```
-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 > 1);
-```
-
-
-```
-UPDATE carbonTable1 SET (c2, c5 ) = (c2 + 1, concat(c5 , "y" ));
-```
-
-
-```
-UPDATE carbonTable1 d SET (c2, c5 ) = (c2 + 1, "xyx")
-WHERE d.column1 = 'india';
-```
-
-
-```
-UPDATE carbonTable1 d SET (c2, c5 ) = (c2 + 1, "xyx")
-WHERE d.column1 = 'india'
-and EXISTS( SELECT * FROM table3 o WHERE o.column2 > 1);
-```
-
-**The Status Success/Failure shall be captured in the driver log and the client.**
-
-
-## Delete Records from CarbonData Table
-This command allows us to delete records from CarbonData table.
-
-### Syntax
-
-```
-DELETE FROM table_name [WHERE expression];
-```
-
-### Parameter Description
-
-| Parameter | Description |
-|--------------|-----------------------------------------------------------------------|
-| table_name | The name of the Carbon table in which you want to perform the delete. |
-
-
-### Examples
-
-```
-DELETE FROM columncarbonTable1 d WHERE d.column1  = 'china';
-```
-
-```
-DELETE FROM dest WHERE column1 IN ('china', 'USA');
-```
-
-```
-DELETE FROM columncarbonTable1
-WHERE column1 IN (SELECT column11 FROM sourceTable2);
-```
-
-```
-DELETE FROM columncarbonTable1
-WHERE column1 IN (SELECT column11 FROM sourceTable2 WHERE
-column1 = 'USA');
-```
-
-```
-DELETE FROM columncarbonTable1 WHERE column2 >= 4
-```
-
-**The Status Success/Failure shall be captured in the driver log and the client.**

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/src/site/markdown/faq.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/faq.md b/src/site/markdown/faq.md
deleted file mode 100644
index 57ac171..0000000
--- a/src/site/markdown/faq.md
+++ /dev/null
@@ -1,77 +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.
--->
-
-# FAQs
-
-* [What are Bad Records?](#what-are-bad-records)
-* [Where are Bad Records Stored in CarbonData?](#where-are-bad-records-stored-in-carbondata)
-* [How to enable Bad Record Logging?](#how-to-enable-bad-record-logging)
-* [How to ignore the Bad Records?](#how-to-ignore-the-bad-records)
-* [How to specify store location while creating carbon session?](#how-to-specify-store-location-while-creating-carbon-session)
-* [What is Carbon Lock Type?](#what-is-carbon-lock-type)
-* [How to resolve Abstract Method Error?](#how-to-resolve-abstract-method-error)
-
-## What are Bad Records?
-Records that fail to get loaded into the CarbonData due to data type incompatibility or are empty or have incompatible format are classified as Bad Records.
-
-## Where are Bad Records Stored in CarbonData?
-The bad records are stored at the location set in carbon.badRecords.location in carbon.properties file.
-By default **carbon.badRecords.location** specifies the following location ``/opt/Carbon/Spark/badrecords``.
-
-## How to enable Bad Record Logging?
-While loading data we can specify the approach to handle Bad Records. In order to analyse the cause of the Bad Records the parameter ``BAD_RECORDS_LOGGER_ENABLE`` must be set to value ``TRUE``. There are multiple approaches to handle Bad Records which can be specified  by the parameter ``BAD_RECORDS_ACTION``.
-
-- To pad the incorrect values of the csv rows with NULL value and load the data in CarbonData, set the following in the query :
-```
-'BAD_RECORDS_ACTION'='FORCE'
-```
-
-- To write the Bad Records without padding incorrect values with NULL in the raw csv (set in the parameter **carbon.badRecords.location**), set the following in the query :
-```
-'BAD_RECORDS_ACTION'='REDIRECT'
-```
-
-## How to ignore the Bad Records?
-To ignore the Bad Records from getting stored in the raw csv, we need to set the following in the query :
-```
-'BAD_RECORDS_ACTION'='IGNORE'
-```
-
-## How to specify store location while creating carbon session?
-The store location specified while creating carbon session is used by the CarbonData to store the meta data like the schema, dictionary files, dictionary meta data and sort indexes.
-
-Try creating ``carbonsession`` with ``storepath`` specified in the following manner :
-```
-val carbon = SparkSession.builder().config(sc.getConf).getOrCreateCarbonSession(<store_path>)
-```
-Example:
-```
-val carbon = SparkSession.builder().config(sc.getConf).getOrCreateCarbonSession("hdfs://localhost:9000/carbon/store ")
-```
-
-## What is Carbon Lock Type?
-The Apache CarbonData acquires lock on the files to prevent concurrent operation from modifying the same files. The lock can be of the following types depending on the storage location, for HDFS we specify it to be of type HDFSLOCK. By default it is set to type LOCALLOCK.
-The property carbon.lock.type configuration specifies the type of lock to be acquired during concurrent operations on table. This property can be set with the following values :
-- **LOCALLOCK** : This Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently.
-- **HDFSLOCK** : This Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData spark applications are launched and no ZooKeeper is running on cluster and the HDFS supports, file based locking.
-
-## How to resolve Abstract Method Error?
-In order to build CarbonData project it is necessary to specify the spark profile. The spark profile sets the Spark Version. You need to specify the ``spark version`` while using Maven to build project.
-
-

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/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
deleted file mode 100644
index bfbcee4..0000000
--- a/src/site/markdown/file-structure-of-carbondata.md
+++ /dev/null
@@ -1,45 +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.
--->
-
-# CarbonData File Structure
-
-CarbonData files contain groups of data called blocklets, along with all required information like schema, offsets and indices etc, in a file header and footer, co-located in HDFS.
-
-The file footer can be read once to build the indices in memory, which can be utilized for optimizing the scans and processing for all subsequent queries.
-
-### Understanding CarbonData File Structure
-* Block : It would be as same as HDFS block, CarbonData creates one file for each data block, user can specify TABLE_BLOCKSIZE during creation table. Each file contains File Header, Blocklets and File Footer. 
-
-![CarbonData File Structure](../docs/images/carbon_data_file_structure_new.png?raw=true)
-
-* 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 to pages, it is guaranteed to be contiguous in file.
-* Page : It has the data of one column and the number of row is fixed to 32000 size. 
-
-![CarbonData File Format](../docs/images/carbon_data_format_new.png?raw=true)
-
-### Each page contains three types of data
-* Data Page: Contains the encoded data of a column of columns.
-* Row ID Page (optional): Contains the row ID mappings used when the data page is stored as an inverted index.
-* RLE Page (optional): Contains additional metadata used when the data page is RLE coded.
-
-
-

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/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 c5bf6df..0000000
--- a/src/site/markdown/installation-guide.md
+++ /dev/null
@@ -1,200 +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/incubator-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. Copy the `./processing/carbonplugins` folder from CarbonData repository to `$SPARK_HOME/carbonlib/` folder.
-
-    **NOTE**: carbonplugins will contain .kettle folder.
-
-6. Repeat Step 2 to Step 5 in all the nodes of the cluster.
-    
-7. In Spark node[master], configure the properties mentioned in the following table in `$SPARK_HOME/conf/spark-defaults.conf` file.
-
-   | Property | Value | Description |
-   |---------------------------------|-----------------------------------------------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------|
-   | carbon.kettle.home | `$SPARK_HOME/carbonlib/carbonplugins` | Path that will be used by CarbonData internally to create graph for loading the data |
-   | 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. |
-
-8. Add the following properties in `$SPARK_HOME/conf/carbon.properties` file:
-
-   | Property             | Required | Description                                                                            | Example                             | Remark  |
-   |----------------------|----------|----------------------------------------------------------------------------------------|-------------------------------------|---------|
-   | carbon.storelocation | NO       | Location where data CarbonData will create the store and write the data in its own format. | hdfs://HOSTNAME:PORT/Opt/CarbonStore      | Propose to set HDFS directory |
-   | carbon.kettle.home   | YES      | Path that will be used by CarbonData internally to create graph for loading the data.         | `$SPARK_HOME/carbonlib/carbonplugins` |         |
-
-
-9. Verify the installation. For example:
-
-   ```
-   ./spark-shell --master spark://HOSTNAME:PORT --total-executor-cores 2
-   --executor-memory 2G
-   ```
-
-**NOTE**: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.
-
-To get started with CarbonData : [Quick Start](quick-start-guide.md), [DDL Operations on CarbonData](ddl-operation-on-carbondata.md)
-
-## Installing and Configuring CarbonData on "Spark on YARN" Cluster
-
-   This section provides the procedure to install CarbonData on "Spark on YARN" cluster.
-
-### Prerequisites
-   * Hadoop HDFS and Yarn should be installed and running.
-   * Spark should be installed and running in all the clients.
-   * CarbonData user should have permission to access HDFS.
-
-### Procedure
-
-   The following steps are only for Driver Nodes. (Driver nodes are the one which starts the spark context.)
-
-1. [Build the CarbonData](https://github.com/apache/incubator-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 `./processing/carbonplugins` folder from CarbonData repository to `$SPARK_HOME/carbonlib/` folder.
-
-    **NOTE**: carbonplugins will contain .kettle folder.
-
-3. Copy the `./conf/carbon.properties.template` file from CarbonData repository to `$SPARK_HOME/conf/` folder and rename the file to `carbon.properties`.
-
-4. 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/
-    ```
-
-5. 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/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` |
-
-
-6. Add the following properties in `$SPARK_HOME/conf/carbon.properties`:
-
-   | Property | Required | Description | Example | Default Value |
-   |----------------------|----------|----------------------------------------------------------------------------------------|-------------------------------------|---------------|
-   | carbon.storelocation | NO | Location where CarbonData will create the store and write the data in its own format. | hdfs://HOSTNAME:PORT/Opt/CarbonStore | Propose to set HDFS directory|
-   | carbon.kettle.home | YES | Path that will be used by CarbonData internally to create graph for loading the data. | carbondata.tar.gz/carbonlib/carbonplugins |  |
-
-
-7. Verify the installation.
-
-   ```
-     ./bin/spark-shell --master yarn-client --driver-memory 1g
-     --executor-cores 2 --executor-memory 2G
-   ```
-  **NOTE**: Make sure you have permissions for CarbonData JARs and files through which driver and executor will start.
-
-  Getting started with CarbonData : [Quick Start](quick-start-guide.md), [DDL Operations on CarbonData](ddl-operation-on-carbondata.md)
-
-## Query Execution Using CarbonData Thrift Server
-
-### Starting CarbonData Thrift Server.
-
-   a. cd `$SPARK_HOME`
-
-   b. Run the following command to start the CarbonData thrift server.
-
-   ```
-   ./bin/spark-submit --conf spark.sql.hive.thriftServer.singleSession=true
-   --class org.apache.carbondata.spark.thriftserver.CarbonThriftServer
-   $SPARK_HOME/carbonlib/$CARBON_ASSEMBLY_JAR <carbon_store_path>
-   ```
-
-| Parameter | Description | Example |
-|---------------------|-----------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|-----------------------------------------------------------------------------------------------------------|
-| CARBON_ASSEMBLY_JAR | CarbonData assembly jar name present in the `$SPARK_HOME/carbonlib/` folder. | carbondata_2.10-0.1.0-incubating-SNAPSHOT-shade-hadoop2.7.2.jar |
-| carbon_store_path | This is a parameter to the CarbonThriftServer class. This a HDFS path where CarbonData files will be kept. Strongly Recommended to put same as carbon.storelocation parameter of carbon.properties. | `hdfs://<host_name>:port/user/hive/warehouse/carbon.store` |
-
-**Examples**
-   
-   * Start with default memory and executors.
-
-```
-./bin/spark-submit --conf spark.sql.hive.thriftServer.singleSession=true 
---class org.apache.carbondata.spark.thriftserver.CarbonThriftServer 
-$SPARK_HOME/carbonlib
-/carbondata_2.10-0.1.0-incubating-SNAPSHOT-shade-hadoop2.7.2.jar 
-hdfs://<host_name>:port/user/hive/warehouse/carbon.store
-```
-   
-   * Start with Fixed executors and resources.
-
-```
-./bin/spark-submit --conf spark.sql.hive.thriftServer.singleSession=true 
---class org.apache.carbondata.spark.thriftserver.CarbonThriftServer 
---num-executors 3 --driver-memory 20g --executor-memory 250g 
---executor-cores 32 
-/srv/OSCON/BigData/HACluster/install/spark/sparkJdbc/lib
-/carbondata_2.10-0.1.0-incubating-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
-     ./bin/beeline jdbc:hive2://<thrftserver_host>:port
-
-     Example
-     ./bin/beeline jdbc:hive2://10.10.10.10:10000
-```
-

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

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/069b05da/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
deleted file mode 100644
index 8f271e3..0000000
--- a/src/site/markdown/supported-data-types-in-carbondata.md
+++ /dev/null
@@ -1,41 +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 Types
-
-#### CarbonData supports the following data types:
-
-  * Numeric Types
-    * SMALLINT
-    * INT/INTEGER
-    * BIGINT
-    * DOUBLE
-    * DECIMAL
-
-  * Date/Time Types
-    * TIMESTAMP
-    * DATE
-
-  * String Types
-    * STRING
-    * CHAR
-
-  * Complex Types
-    * arrays: ARRAY``<data_type>``
-    * structs: STRUCT``<col_name : data_type COMMENT col_comment, ...>``


[6/9] carbondata-site git commit: Merge pull request #10 from jatin9896/feature/PdfFunctionality

Posted by ch...@apache.org.
Merge pull request #10 from jatin9896/feature/PdfFunctionality

Feature/pdf functionality

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

Branch: refs/heads/asf-site
Commit: 3a8c7ff8e351e1808397307f1085c27d10b8ab95
Parents: cf9ebc6 069b05d
Author: Pallavi Singh <pa...@yahoo.co.in>
Authored: Tue May 23 12:32:26 2017 +0000
Committer: GitHub <no...@github.com>
Committed: Tue May 23 12:32:26 2017 +0000

----------------------------------------------------------------------
 carbonscript.sh                                 |   2 +
 content/pdf/maven-pdf-plugin.pdf                | Bin 145272 -> 0 bytes
 pom.xml                                         |  89 +++-
 src/main/resources/application.conf             |   3 +-
 src/main/scala/MDFileConverter.scala            |   1 +
 src/site/images/CarbonData_logo.png             | Bin 0 -> 10971 bytes
 .../images/carbon_data_file_structure_new.png   | Bin 0 -> 9477 bytes
 src/site/images/carbon_data_format_new.png      | Bin 0 -> 35510 bytes
 src/site/pdf.xml                                |  35 ++
 src/site/projectLogo/ApacheLogo.png             | Bin 0 -> 7241 bytes
 src/site/projectLogo/CarbonDataLogo.png         | Bin 0 -> 10971 bytes
 src/site/resources/pdf-config.xml               | 520 +++++++++++++++++++
 12 files changed, 648 insertions(+), 2 deletions(-)
----------------------------------------------------------------------