You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ch...@apache.org on 2018/06/02 03:27:28 UTC

[1/5] carbondata-site git commit: doucment for latest release of carbondata 1.4.0

Repository: carbondata-site
Updated Branches:
  refs/heads/asf-site 8cb979c8d -> 61436c8e4


http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/site/markdown/data-management-on-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/data-management-on-carbondata.md b/src/site/markdown/data-management-on-carbondata.md
index 51e98ab..706209c 100644
--- a/src/site/markdown/data-management-on-carbondata.md
+++ b/src/site/markdown/data-management-on-carbondata.md
@@ -35,11 +35,11 @@ This tutorial is going to introduce all commands and data operations on CarbonDa
   
   ```
   CREATE TABLE [IF NOT EXISTS] [db_name.]table_name[(col_name data_type , ...)]
-  STORED BY 'carbondata'
+  STORED AS carbondata
   [TBLPROPERTIES (property_name=property_value, ...)]
   [LOCATION 'path']
   ```
-  **NOTE:** CarbonData also supports "STORED AS carbondata". Find example code at [CarbonSessionExample](https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala) in the CarbonData repo.
+  **NOTE:** CarbonData also supports "STORED AS carbondata" and "USING carbondata". Find example code at [CarbonSessionExample](https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/scala/org/apache/carbondata/examples/CarbonSessionExample.scala) in the CarbonData repo.
 ### Usage Guidelines
 
   Following are the guidelines for TBLPROPERTIES, CarbonData's additional table options can be set via carbon.properties.

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/site/markdown/lucene-datamap-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/lucene-datamap-guide.md b/src/site/markdown/lucene-datamap-guide.md
new file mode 100644
index 0000000..5f7a2e4
--- /dev/null
+++ b/src/site/markdown/lucene-datamap-guide.md
@@ -0,0 +1,159 @@
+# CarbonData Lucene DataMap (Alpha feature in 1.4.0)
+  
+* [DataMap Management](#datamap-management)
+* [Lucene Datamap](#lucene-datamap-introduction)
+* [Loading Data](#loading-data)
+* [Querying Data](#querying-data)
+* [Data Management](#data-management-with-lucene-datamap)
+
+#### DataMap Management 
+Lucene DataMap can be created using following DDL
+  ```
+  CREATE DATAMAP [IF NOT EXISTS] datamap_name
+  ON TABLE main_table
+  USING 'lucene'
+  DMPROPERTIES ('index_columns'='city, name', ...)
+  ```
+
+DataMap can be dropped using following DDL:
+  ```
+  DROP DATAMAP [IF EXISTS] datamap_name
+  ON TABLE main_table
+  ```
+To show all DataMaps created, use:
+  ```
+  SHOW DATAMAP 
+  ON TABLE main_table
+  ```
+It will show all DataMaps created on main table.
+
+
+## Lucene DataMap Introduction
+  Lucene is a high performance, full featured text search engine. Lucene is integrated to carbon as
+  an index datamap and managed along with main tables by CarbonData.User can create lucene datamap 
+  to improve query performance on string columns which has content of more length. So, user can 
+  search tokenized word or pattern of it using lucene query on text content.
+  
+  For instance, main table called **datamap_test** which is defined as:
+  
+  ```
+  CREATE TABLE datamap_test (
+    name string,
+    age int,
+    city string,
+    country string)
+  STORED BY 'carbondata'
+  ```
+  
+  User can create Lucene datamap using the Create DataMap DDL:
+  
+  ```
+  CREATE DATAMAP dm
+  ON TABLE datamap_test
+  USING 'lucene'
+  DMPROPERTIES ('INDEX_COLUMNS' = 'name, country',)
+  ```
+
+**DMProperties**
+1. INDEX_COLUMNS: The list of string columns on which lucene creates indexes.
+2. FLUSH_CACHE: size of the cache to maintain in Lucene writer, if specified then it tries to 
+   aggregate the unique data till the cache limit and flush to Lucene. It is best suitable for low 
+   cardinality dimensions.
+3. SPLIT_BLOCKLET: when made as true then store the data in blocklet wise in lucene , it means new 
+   folder will be created for each blocklet, thus, it eliminates storing blockletid in lucene and 
+   also it makes lucene small chunks of data.
+   
+## Loading data
+When loading data to main table, lucene index files will be generated for all the
+index_columns(String Columns) given in DMProperties which contains information about the data
+location of index_columns. These index files will be written inside a folder named with datamap name
+inside each segment folders.
+
+A system level configuration carbon.lucene.compression.mode can be added for best compression of
+lucene index files. The default value is speed, where the index writing speed will be more. If the
+value is compression, the index file size will be compressed.
+
+## Querying data
+As a technique for query acceleration, Lucene indexes cannot be queried directly.
+Queries are to be made on main table. when a query with TEXT_MATCH('name:c10') or 
+TEXT_MATCH_WITH_LIMIT('name:n10',10)[the second parameter represents the number of result to be 
+returned, if user does not specify this value, all results will be returned without any limit] is 
+fired, two jobs are fired.The first job writes the temporary files in folder created at table level 
+which contains lucene's seach results and these files will be read in second job to give faster 
+results. These temporary files will be cleared once the query finishes.
+
+User can verify whether a query can leverage Lucene datamap or not by executing `EXPLAIN`
+command, which will show the transformed logical plan, and thus user can check whether TEXT_MATCH()
+filter is applied on query or not.
+
+**Note:**
+ 1. The filter columns in TEXT_MATCH or TEXT_MATCH_WITH_LIMIT must be always in lower case and 
+filter condition like 'AND','OR' must be in upper case.
+
+      Ex: 
+      ```
+      select * from datamap_test where TEXT_MATCH('name:*10 AND name:*n*')
+      ```
+     
+2. Query supports only one TEXT_MATCH udf for filter condition and not multiple udfs.
+
+   The following query is supported:
+   ```
+   select * from datamap_test where TEXT_MATCH('name:*10 AND name:*n*')
+   ```
+       
+   The following query is not supported:
+   ```
+   select * from datamap_test where TEXT_MATCH('name:*10) AND TEXT_MATCH(name:*n*')
+   ```
+       
+          
+Below like queries can be converted to text_match queries as following:
+```
+select * from datamap_test where name='n10'
+
+select * from datamap_test where name like 'n1%'
+
+select * from datamap_test where name like '%10'
+
+select * from datamap_test where name like '%n%'
+
+select * from datamap_test where name like '%10' and name not like '%n%'
+```
+Lucene TEXT_MATCH Queries:
+```
+select * from datamap_test where TEXT_MATCH('name:n10')
+
+select * from datamap_test where TEXT_MATCH('name:n1*')
+
+select * from datamap_test where TEXT_MATCH('name:*10')
+
+select * from datamap_test where TEXT_MATCH('name:*n*')
+
+select * from datamap_test where TEXT_MATCH('name:*10 -name:*n*')
+```
+**Note:** For lucene queries and syntax, refer to [lucene-syntax](www.lucenetutorial.com/lucene-query-syntax.html)
+
+## Data Management with lucene datamap
+Once there is lucene datamap is created on the main table, following command on the main
+table
+is not supported:
+1. Data management command: `UPDATE/DELETE`.
+2. Schema management command: `ALTER TABLE DROP COLUMN`, `ALTER TABLE CHANGE DATATYPE`, 
+`ALTER TABLE RENAME`.
+
+**Note**: Adding a new column is supported, and for dropping columns and change datatype 
+command, CarbonData will check whether it will impact the lucene datamap, if not, the operation 
+is allowed, otherwise operation will be rejected by throwing exception.
+
+
+3. Partition management command: `ALTER TABLE ADD/DROP PARTITION`.
+
+However, there is still way to support these operations on main table, in current CarbonData 
+release, user can do as following:
+1. Remove the lucene datamap by `DROP DATAMAP` command.
+2. Carry out the data management operation on main table.
+3. Create the lucene datamap again by `CREATE DATAMAP` command.
+Basically, user can manually trigger the operation by re-building the datamap.
+
+

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/site/markdown/sdk-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/sdk-guide.md b/src/site/markdown/sdk-guide.md
new file mode 100644
index 0000000..1d225a9
--- /dev/null
+++ b/src/site/markdown/sdk-guide.md
@@ -0,0 +1,680 @@
+# SDK Guide
+In the carbon jars package, there exist a carbondata-store-sdk-x.x.x-SNAPSHOT.jar, including SDK writer and reader.
+# SDK Writer
+This SDK writer, writes carbondata file and carbonindex file at a given path.
+External client can make use of this writer to convert other format data or live data to create carbondata and index files.
+These SDK writer output contains just a carbondata and carbonindex files. No metadata folder will be present.
+
+## Quick example
+
+### Example with csv format 
+
+```java
+ import java.io.IOException;
+ 
+ import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
+ import org.apache.carbondata.core.metadata.datatype.DataTypes;
+ import org.apache.carbondata.core.util.CarbonProperties;
+ import org.apache.carbondata.sdk.file.CarbonWriter;
+ import org.apache.carbondata.sdk.file.CarbonWriterBuilder;
+ import org.apache.carbondata.sdk.file.Field;
+ import org.apache.carbondata.sdk.file.Schema;
+ 
+ public class TestSdk {
+
+   // pass true or false while executing the main to use offheap memory or not
+   public static void main(String[] args) throws IOException, InvalidLoadOptionException {
+     if (args.length > 0 && args[0] != null) {
+       testSdkWriter(args[0]);
+     } else {
+       testSdkWriter("true");
+     }
+   }
+ 
+   public static void testSdkWriter(String enableOffheap) throws IOException, InvalidLoadOptionException {
+     String path = "./target/testCSVSdkWriter";
+ 
+     Field[] fields = new Field[2];
+     fields[0] = new Field("name", DataTypes.STRING);
+     fields[1] = new Field("age", DataTypes.INT);
+ 
+     Schema schema = new Schema(fields);
+
+     CarbonProperties.getInstance().addProperty("enable.offheap.sort", enableOffheap);
+ 
+     CarbonWriterBuilder builder = CarbonWriter.builder().outputPath(path);
+ 
+     CarbonWriter writer = builder.buildWriterForCSVInput(schema);
+ 
+     int rows = 5;
+     for (int i = 0; i < rows; i++) {
+       writer.write(new String[] { "robot" + (i % 10), String.valueOf(i) });
+     }
+     writer.close();
+   }
+ }
+```
+
+### Example with Avro format
+```java
+import java.io.IOException;
+
+import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.sdk.file.AvroCarbonWriter;
+import org.apache.carbondata.sdk.file.CarbonWriter;
+import org.apache.carbondata.sdk.file.Field;
+
+import org.apache.avro.generic.GenericData;
+import org.apache.commons.lang.CharEncoding;
+
+import tech.allegro.schema.json2avro.converter.JsonAvroConverter;
+
+public class TestSdkAvro {
+
+  public static void main(String[] args) throws IOException, InvalidLoadOptionException {
+    testSdkWriter();
+  }
+
+
+  public static void testSdkWriter() throws IOException, InvalidLoadOptionException {
+    String path = "./AvroCarbonWriterSuiteWriteFiles";
+    // Avro schema
+    String avroSchema =
+        "{" +
+            "   \"type\" : \"record\"," +
+            "   \"name\" : \"Acme\"," +
+            "   \"fields\" : ["
+            + "{ \"name\" : \"fname\", \"type\" : \"string\" },"
+            + "{ \"name\" : \"age\", \"type\" : \"int\" }]" +
+            "}";
+
+    String json = "{\"fname\":\"bob\", \"age\":10}";
+
+    // conversion to GenericData.Record
+    JsonAvroConverter converter = new JsonAvroConverter();
+    GenericData.Record record = converter.convertToGenericDataRecord(
+        json.getBytes(CharEncoding.UTF_8), new org.apache.avro.Schema.Parser().parse(avroSchema));
+
+    try {
+      CarbonWriter writer = CarbonWriter.builder()
+          .outputPath(path)
+          .buildWriterForAvroInput(new org.apache.avro.Schema.Parser().parse(avroSchema));
+
+      for (int i = 0; i < 100; i++) {
+        writer.write(record);
+      }
+      writer.close();
+    } catch (Exception e) {
+      e.printStackTrace();
+    }
+  }
+}
+```
+
+## Datatypes Mapping
+Each of SQL data types are mapped into data types of SDK. Following are the mapping:
+
+| SQL DataTypes | Mapped SDK DataTypes |
+|---------------|----------------------|
+| BOOLEAN | DataTypes.BOOLEAN |
+| SMALLINT | DataTypes.SHORT |
+| INTEGER | DataTypes.INT |
+| BIGINT | DataTypes.LONG |
+| DOUBLE | DataTypes.DOUBLE |
+| VARCHAR | DataTypes.STRING |
+| DATE | DataTypes.DATE |
+| TIMESTAMP | DataTypes.TIMESTAMP |
+| STRING | DataTypes.STRING |
+| DECIMAL | DataTypes.createDecimalType(precision, scale) |
+
+
+## API List
+
+### Class org.apache.carbondata.sdk.file.CarbonWriterBuilder
+```
+/**
+* Sets the output path of the writer builder
+* @param path is the absolute path where output files are written
+*             This method must be called when building CarbonWriterBuilder
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder outputPath(String path);
+```
+
+```
+/**
+* If set false, writes the carbondata and carbonindex files in a flat folder structure
+* @param isTransactionalTable is a boolelan value
+*             if set to false, then writes the carbondata and carbonindex files
+*                                                            in a flat folder structure.
+*             if set to true, then writes the carbondata and carbonindex files
+*                                                            in segment folder structure..
+*             By default set to false.
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder isTransactionalTable(boolean isTransactionalTable);
+```
+
+```
+/**
+* to set the timestamp in the carbondata and carbonindex index files
+* @param UUID is a timestamp to be used in the carbondata and carbonindex index files.
+*             By default set to zero.
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder uniqueIdentifier(long UUID);
+```
+
+```
+/**
+* To set the carbondata file size in MB between 1MB-2048MB
+* @param blockSize is size in MB between 1MB to 2048 MB
+*                  default value is 1024 MB
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder withBlockSize(int blockSize);
+```
+
+```
+/**
+* To set the blocklet size of carbondata file
+* @param blockletSize is blocklet size in MB
+*                     default value is 64 MB
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder withBlockletSize(int blockletSize);
+```
+
+```
+/**
+* sets the list of columns that needs to be in sorted order
+* @param sortColumns is a string array of columns that needs to be sorted.
+*                    If it is null or by default all dimensions are selected for sorting
+*                    If it is empty array, no columns are sorted
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder sortBy(String[] sortColumns);
+```
+
+```
+/**
+* If set, create a schema file in metadata folder.
+* @param persist is a boolean value, If set to true, creates a schema file in metadata folder.
+*                By default set to false. will not create metadata folder
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder persistSchemaFile(boolean persist);
+```
+
+```
+/**
+* sets the taskNo for the writer. SDKs concurrently running
+* will set taskNo in order to avoid conflicts in file's name during write.
+* @param taskNo is the TaskNo user wants to specify.
+*               by default it is system time in nano seconds.
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder taskNo(String taskNo);
+```
+
+```
+/**
+* To support the load options for sdk writer
+* @param options key,value pair of load options.
+*                supported keys values are
+*                a. bad_records_logger_enable -- true (write into separate logs), false
+*                b. bad_records_action -- FAIL, FORCE, IGNORE, REDIRECT
+*                c. bad_record_path -- path
+*                d. dateformat -- same as JAVA SimpleDateFormat
+*                e. timestampformat -- same as JAVA SimpleDateFormat
+*                f. complex_delimiter_level_1 -- value to Split the complexTypeData
+*                g. complex_delimiter_level_2 -- value to Split the nested complexTypeData
+*                h. quotechar
+*                i. escapechar
+*
+*                Default values are as follows.
+*
+*                a. bad_records_logger_enable -- "false"
+*                b. bad_records_action -- "FAIL"
+*                c. bad_record_path -- ""
+*                d. dateformat -- "" , uses from carbon.properties file
+*                e. timestampformat -- "", uses from carbon.properties file
+*                f. complex_delimiter_level_1 -- "$"
+*                g. complex_delimiter_level_2 -- ":"
+*                h. quotechar -- "\""
+*                i. escapechar -- "\\"
+*
+* @return updated CarbonWriterBuilder
+*/
+public CarbonWriterBuilder withLoadOptions(Map<String, String> options);
+```
+
+```
+/**
+* Build a {@link CarbonWriter}, which accepts row in CSV format object
+* @param schema carbon Schema object {org.apache.carbondata.sdk.file.Schema}
+* @return CSVCarbonWriter
+* @throws IOException
+* @throws InvalidLoadOptionException
+*/
+public CarbonWriter buildWriterForCSVInput() throws IOException, InvalidLoadOptionException;
+```
+
+```  
+/**
+* Build a {@link CarbonWriter}, which accepts Avro format object
+* @param avroSchema avro Schema object {org.apache.avro.Schema}
+* @return AvroCarbonWriter 
+* @throws IOException
+* @throws InvalidLoadOptionException
+*/
+public CarbonWriter buildWriterForAvroInput() throws IOException, InvalidLoadOptionException;
+```
+
+### Class org.apache.carbondata.sdk.file.CarbonWriter
+```
+/**
+* Write an object to the file, the format of the object depends on the implementation
+* If AvroCarbonWriter, object is of type org.apache.avro.generic.GenericData.Record 
+* If CSVCarbonWriter, object is of type String[]
+* Note: This API is not thread safe
+* @param object
+* @throws IOException
+*/
+public abstract void write(Object object) throws IOException;
+```
+
+```
+/**
+* Flush and close the writer
+*/
+public abstract void close() throws IOException;
+```
+
+```
+/**
+* Create a {@link CarbonWriterBuilder} to build a {@link CarbonWriter}
+*/
+public static CarbonWriterBuilder builder() {
+    return new CarbonWriterBuilder();
+}
+```
+
+### Class org.apache.carbondata.sdk.file.Field
+```
+/**
+* Field Constructor
+* @param name name of the field
+* @param type datatype of field, specified in strings.
+*/
+public Field(String name, String type);
+```
+
+```
+/**
+* Field constructor
+* @param name name of the field
+* @param type datatype of the field of class DataType
+*/
+public Field(String name, DataType type);  
+```
+
+### Class org.apache.carbondata.sdk.file.Schema
+
+```
+/**
+* construct a schema with fields
+* @param fields
+*/
+public Schema(Field[] fields);
+```
+
+```
+/**
+* Create a Schema using JSON string, for example:
+* [
+*   {"name":"string"},
+*   {"age":"int"}
+* ] 
+* @param json specified as string
+* @return Schema
+*/
+public static Schema parseJson(String json);
+```
+
+### Class org.apache.carbondata.core.util.CarbonProperties
+
+```
+/**
+* This method will be responsible to get the instance of CarbonProperties class
+*
+* @return carbon properties instance
+*/
+public static CarbonProperties getInstance();
+```
+
+```
+/**
+* This method will be used to add a new property
+*
+* @param key is a property name to set for carbon.
+* @param value is valid parameter corresponding to property.
+* @return CarbonProperties object
+*/
+public CarbonProperties addProperty(String key, String value);
+```
+
+```
+/**
+* This method will be used to get the property value. If property is not
+* present, then it will return the default value.
+*
+* @param key is a property name to get user specified value.
+* @return properties value for corresponding key. If not set, then returns null.
+*/
+public String getProperty(String key);
+```
+
+```
+/**
+* This method will be used to get the property value. If property is not
+* present, then it will return the default value.
+*
+* @param key is a property name to get user specified value..
+* @param defaultValue used to be returned by function if corrosponding key not set.
+* @return properties value for corresponding key. If not set, then returns specified defaultValue.
+*/
+public String getProperty(String key, String defaultValue);
+```
+Reference : [list of carbon properties](http://carbondata.apache.org/configuration-parameters.html)
+
+### Class org.apache.carbondata.sdk.file.AvroCarbonWriter
+```
+/**
+* converts avro schema to carbon schema, required by carbonWriter
+*
+* @param avroSchemaString json formatted avro schema as string
+* @return carbon sdk schema
+*/
+public static org.apache.carbondata.sdk.file.Schema getCarbonSchemaFromAvroSchema(String avroSchemaString);
+```
+# SDK Reader
+This SDK reader reads CarbonData file and carbonindex file at a given path.
+External client can make use of this reader to read CarbonData files without CarbonSession.
+## Quick example
+```
+    // 1. Create carbon reader
+    String path = "./testWriteFiles";
+    CarbonReader reader = CarbonReader
+        .builder(path, "_temp")
+        .projection(new String[]{"name", "age"})
+        .build();
+
+    // 2. Read data
+    int i = 0;
+    while (reader.hasNext()) {
+      Object[] row = (Object[]) reader.readNextRow();
+      System.out.println(row[0] + "\t" + row[1]);
+      i++;
+    }
+    
+    // 3. Close this reader
+    reader.close();
+```
+
+Find example code at [CarbonReaderExample](https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/CarbonReaderExample.java) in the CarbonData repo.
+
+## API List
+
+### Class org.apache.carbondata.sdk.file.CarbonReader
+```
+ /**
+  * Return a new CarbonReaderBuilder instance
+  */
+  public static CarbonReaderBuilder builder(String tablePath, String tableName);
+```
+
+```
+  /**
+   * Return true if has next row
+   */
+  public boolean hasNext();
+```
+
+```
+  /**
+   * Read and return next row object
+   */
+  public T readNextRow();
+```
+
+```
+  /**
+   * Close reader
+   */
+  public void close();
+```
+
+### Class org.apache.carbondata.sdk.file.CarbonReaderBuilder
+```
+  /**
+   * Construct a CarbonReaderBuilder with table path and table name
+   *
+   * @param tablePath table path
+   * @param tableName table name
+   */
+  CarbonReaderBuilder(String tablePath, String tableName);
+```
+
+```
+  /**
+   * Configure the projection column names of carbon reader
+   *
+   * @param projectionColumnNames projection column names
+   * @return CarbonReaderBuilder object
+   */
+  public CarbonReaderBuilder projection(String[] projectionColumnNames);
+```
+
+```
+  /**
+   * Project all Columns for carbon reader
+   *
+   * @return CarbonReaderBuilder object
+   * @throws IOException
+   */
+  public CarbonReaderBuilder projectAllColumns();
+```
+
+```
+  /**
+   * Configure the transactional status of table
+   * If set to false, then reads the carbondata and carbonindex files from a flat folder structure.
+   * If set to true, then reads the carbondata and carbonindex files from segment folder structure.
+   * Default value is false
+   *
+   * @param isTransactionalTable whether is transactional table or not
+   * @return CarbonReaderBuilder object
+   */
+  public CarbonReaderBuilder isTransactionalTable(boolean isTransactionalTable);
+```
+
+```
+ /**
+  * Configure the filter expression for carbon reader
+  *
+  * @param filterExpression filter expression
+  * @return CarbonReaderBuilder object
+  */
+  public CarbonReaderBuilder filter(Expression filterExpression);
+```
+
+```
+  /**
+   * Set the access key for S3
+   *
+   * @param key   the string of access key for different S3 type,like: fs.s3a.access.key
+   * @param value the value of access key
+   * @return CarbonWriterBuilder
+   */
+  public CarbonReaderBuilder setAccessKey(String key, String value);
+```
+
+```
+  /**
+   * Set the access key for S3.
+   *
+   * @param value the value of access key
+   * @return CarbonWriterBuilder object
+   */
+  public CarbonReaderBuilder setAccessKey(String value);
+```
+
+```
+  /**
+   * Set the secret key for S3
+   *
+   * @param key   the string of secret key for different S3 type,like: fs.s3a.secret.key
+   * @param value the value of secret key
+   * @return CarbonWriterBuilder object
+   */
+  public CarbonReaderBuilder setSecretKey(String key, String value);
+```
+
+```
+  /**
+   * Set the secret key for S3
+   *
+   * @param value the value of secret key
+   * @return CarbonWriterBuilder object
+   */
+  public CarbonReaderBuilder setSecretKey(String value);
+```
+
+```
+ /**
+   * Set the endpoint for S3
+   *
+   * @param key   the string of endpoint for different S3 type,like: fs.s3a.endpoint
+   * @param value the value of endpoint
+   * @return CarbonWriterBuilder object
+   */
+  public CarbonReaderBuilder setEndPoint(String key, String value);
+```
+
+``` 
+  /**
+   * Set the endpoint for S3
+   *
+   * @param value the value of endpoint
+   * @return CarbonWriterBuilder object
+   */
+  public CarbonReaderBuilder setEndPoint(String value);
+```
+
+```
+ /**
+   * Build CarbonReader
+   *
+   * @param <T>
+   * @return CarbonReader
+   * @throws IOException
+   * @throws InterruptedException
+   */
+  public <T> CarbonReader<T> build();
+```
+### Class org.apache.carbondata.sdk.file.CarbonSchemaReader
+```
+  /**
+   * Read schema file and return the schema
+   *
+   * @param schemaFilePath complete path including schema file name
+   * @return schema object
+   * @throws IOException
+   */
+  public static Schema readSchemaInSchemaFile(String schemaFilePath);
+```
+
+```
+  /**
+   * Read carbondata file and return the schema
+   *
+   * @param dataFilePath complete path including carbondata file name
+   * @return Schema object
+   * @throws IOException
+   */
+  public static Schema readSchemaInDataFile(String dataFilePath);
+```
+
+```
+  /**
+   * Read carbonindex file and return the schema
+   *
+   * @param indexFilePath complete path including index file name
+   * @return schema object
+   * @throws IOException
+   */
+  public static Schema readSchemaInIndexFile(String indexFilePath);
+```
+
+### Class org.apache.carbondata.sdk.file.Schema
+```
+  /**
+   * construct a schema with fields
+   * @param fields
+   */
+  public Schema(Field[] fields);
+```
+
+```
+  /**
+   * construct a schema with List<ColumnSchema>
+   *
+   * @param columnSchemaList column schema list
+   */
+  public Schema(List<ColumnSchema> columnSchemaList);
+```
+
+```
+  /**
+   * Create a Schema using JSON string, for example:
+   * [
+   *   {"name":"string"},
+   *   {"age":"int"}
+   * ]
+   * @param json specified as string
+   * @return Schema
+   */
+  public static Schema parseJson(String json);
+```
+
+```
+  /**
+   * Sort the schema order as original order
+   *
+   * @return Schema object
+   */
+  public Schema asOriginOrder();
+```
+
+### Class org.apache.carbondata.sdk.file.Field
+```
+  /**
+   * Field Constructor
+   * @param name name of the field
+   * @param type datatype of field, specified in strings.
+   */
+  public Field(String name, String type);
+```
+
+```
+  /**
+   * Construct Field from ColumnSchema
+   *
+   * @param columnSchema ColumnSchema, Store the information about the column meta data
+   */
+  public Field(ColumnSchema columnSchema);
+```
+
+Find S3 example code at [SDKS3Example](https://github.com/apache/carbondata/blob/master/examples/spark2/src/main/java/org/apache/carbondata/examples/sdk/SDKS3Example.java) in the CarbonData repo.

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/site/markdown/sdk-writer-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/sdk-writer-guide.md b/src/site/markdown/sdk-writer-guide.md
deleted file mode 100644
index 9878b71..0000000
--- a/src/site/markdown/sdk-writer-guide.md
+++ /dev/null
@@ -1,359 +0,0 @@
-# SDK Writer Guide
-In the carbon jars package, there exist a carbondata-store-sdk-x.x.x-SNAPSHOT.jar.
-This SDK writer, writes carbondata file and carbonindex file at a given path.
-External client can make use of this writer to convert other format data or live data to create carbondata and index files.
-These SDK writer output contains just a carbondata and carbonindex files. No metadata folder will be present.
-
-## Quick example
-
-### Example with csv format 
-
-```java
- import java.io.IOException;
- 
- import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
- import org.apache.carbondata.core.metadata.datatype.DataTypes;
- import org.apache.carbondata.sdk.file.CarbonWriter;
- import org.apache.carbondata.sdk.file.CarbonWriterBuilder;
- import org.apache.carbondata.sdk.file.Field;
- import org.apache.carbondata.sdk.file.Schema;
- 
- public class TestSdk {
- 
-   public static void main(String[] args) throws IOException, InvalidLoadOptionException {
-     testSdkWriter();
-   }
- 
-   public static void testSdkWriter() throws IOException, InvalidLoadOptionException {
-     String path = "/home/root1/Documents/ab/temp";
- 
-     Field[] fields = new Field[2];
-     fields[0] = new Field("name", DataTypes.STRING);
-     fields[1] = new Field("age", DataTypes.INT);
- 
-     Schema schema = new Schema(fields);
- 
-     CarbonWriterBuilder builder = CarbonWriter.builder().withSchema(schema).outputPath(path);
- 
-     CarbonWriter writer = builder.buildWriterForCSVInput();
- 
-     int rows = 5;
-     for (int i = 0; i < rows; i++) {
-       writer.write(new String[] { "robot" + (i % 10), String.valueOf(i) });
-     }
-     writer.close();
-   }
- }
-```
-
-### Example with Avro format
-```java
-import java.io.IOException;
-
-import org.apache.carbondata.common.exceptions.sql.InvalidLoadOptionException;
-import org.apache.carbondata.core.metadata.datatype.DataTypes;
-import org.apache.carbondata.sdk.file.AvroCarbonWriter;
-import org.apache.carbondata.sdk.file.CarbonWriter;
-import org.apache.carbondata.sdk.file.Field;
-
-import org.apache.avro.generic.GenericData;
-import org.apache.commons.lang.CharEncoding;
-
-import tech.allegro.schema.json2avro.converter.JsonAvroConverter;
-
-public class TestSdkAvro {
-
-  public static void main(String[] args) throws IOException, InvalidLoadOptionException {
-    testSdkWriter();
-  }
-
-
-  public static void testSdkWriter() throws IOException, InvalidLoadOptionException {
-    String path = "./AvroCarbonWriterSuiteWriteFiles";
-    // Avro schema
-    String avroSchema =
-        "{" +
-            "   \"type\" : \"record\"," +
-            "   \"name\" : \"Acme\"," +
-            "   \"fields\" : ["
-            + "{ \"name\" : \"fname\", \"type\" : \"string\" },"
-            + "{ \"name\" : \"age\", \"type\" : \"int\" }]" +
-            "}";
-
-    String json = "{\"fname\":\"bob\", \"age\":10}";
-
-    // conversion to GenericData.Record
-    JsonAvroConverter converter = new JsonAvroConverter();
-    GenericData.Record record = converter.convertToGenericDataRecord(
-        json.getBytes(CharEncoding.UTF_8), new org.apache.avro.Schema.Parser().parse(avroSchema));
-
-    // prepare carbon schema from avro schema 
-    org.apache.carbondata.sdk.file.Schema carbonSchema =
-            AvroCarbonWriter.getCarbonSchemaFromAvroSchema(avroSchema);
-
-    try {
-      CarbonWriter writer = CarbonWriter.builder()
-          .withSchema(carbonSchema)
-          .outputPath(path)
-          .buildWriterForAvroInput();
-
-      for (int i = 0; i < 100; i++) {
-        writer.write(record);
-      }
-      writer.close();
-    } catch (Exception e) {
-      e.printStackTrace();
-    }
-  }
-}
-```
-
-## Datatypes Mapping
-Each of SQL data types are mapped into data types of SDK. Following are the mapping:
-
-| SQL DataTypes | Mapped SDK DataTypes |
-|---------------|----------------------|
-| BOOLEAN | DataTypes.BOOLEAN |
-| SMALLINT | DataTypes.SHORT |
-| INTEGER | DataTypes.INT |
-| BIGINT | DataTypes.LONG |
-| DOUBLE | DataTypes.DOUBLE |
-| VARCHAR | DataTypes.STRING |
-| DATE | DataTypes.DATE |
-| TIMESTAMP | DataTypes.TIMESTAMP |
-| STRING | DataTypes.STRING |
-| DECIMAL | DataTypes.createDecimalType(precision, scale) |
-
-
-## API List
-
-### Class org.apache.carbondata.sdk.file.CarbonWriterBuilder
-```
-/**
-* prepares the builder with the schema provided
-* @param schema is instance of Schema
-*        This method must be called when building CarbonWriterBuilder
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder withSchema(Schema schema);
-```
-
-```
-/**
-* Sets the output path of the writer builder
-* @param path is the absolute path where output files are written
-*             This method must be called when building CarbonWriterBuilder
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder outputPath(String path);
-```
-
-```
-/**
-* If set false, writes the carbondata and carbonindex files in a flat folder structure
-* @param isTransactionalTable is a boolelan value
-*             if set to false, then writes the carbondata and carbonindex files
-*                                                            in a flat folder structure.
-*             if set to true, then writes the carbondata and carbonindex files
-*                                                            in segment folder structure..
-*             By default set to false.
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder isTransactionalTable(boolean isTransactionalTable);
-```
-
-```
-/**
-* to set the timestamp in the carbondata and carbonindex index files
-* @param UUID is a timestamp to be used in the carbondata and carbonindex index files.
-*             By default set to zero.
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder uniqueIdentifier(long UUID);
-```
-
-```
-/**
-* To set the carbondata file size in MB between 1MB-2048MB
-* @param blockSize is size in MB between 1MB to 2048 MB
-*                  default value is 1024 MB
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder withBlockSize(int blockSize);
-```
-
-```
-/**
-* To set the blocklet size of carbondata file
-* @param blockletSize is blocklet size in MB
-*                     default value is 64 MB
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder withBlockletSize(int blockletSize);
-```
-
-```
-/**
-* sets the list of columns that needs to be in sorted order
-* @param sortColumns is a string array of columns that needs to be sorted.
-*                    If it is null or by default all dimensions are selected for sorting
-*                    If it is empty array, no columns are sorted
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder sortBy(String[] sortColumns);
-```
-
-```
-/**
-* If set, create a schema file in metadata folder.
-* @param persist is a boolean value, If set to true, creates a schema file in metadata folder.
-*                By default set to false. will not create metadata folder
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder persistSchemaFile(boolean persist);
-```
-
-```
-/**
-* sets the taskNo for the writer. SDKs concurrently running
-* will set taskNo in order to avoid conflicts in file's name during write.
-* @param taskNo is the TaskNo user wants to specify.
-*               by default it is system time in nano seconds.
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder taskNo(String taskNo);
-```
-
-```
-/**
-* To support the load options for sdk writer
-* @param options key,value pair of load options.
-*                supported keys values are
-*                a. bad_records_logger_enable -- true (write into separate logs), false
-*                b. bad_records_action -- FAIL, FORCE, IGNORE, REDIRECT
-*                c. bad_record_path -- path
-*                d. dateformat -- same as JAVA SimpleDateFormat
-*                e. timestampformat -- same as JAVA SimpleDateFormat
-*                f. complex_delimiter_level_1 -- value to Split the complexTypeData
-*                g. complex_delimiter_level_2 -- value to Split the nested complexTypeData
-*                h. quotechar
-*                i. escapechar
-*
-*                Default values are as follows.
-*
-*                a. bad_records_logger_enable -- "false"
-*                b. bad_records_action -- "FAIL"
-*                c. bad_record_path -- ""
-*                d. dateformat -- "" , uses from carbon.properties file
-*                e. timestampformat -- "", uses from carbon.properties file
-*                f. complex_delimiter_level_1 -- "$"
-*                g. complex_delimiter_level_2 -- ":"
-*                h. quotechar -- "\""
-*                i. escapechar -- "\\"
-*
-* @return updated CarbonWriterBuilder
-*/
-public CarbonWriterBuilder withLoadOptions(Map<String, String> options);
-```
-
-```
-/**
-* Build a {@link CarbonWriter}, which accepts row in CSV format object
-* @return CSVCarbonWriter
-* @throws IOException
-* @throws InvalidLoadOptionException
-*/
-public CarbonWriter buildWriterForCSVInput() throws IOException, InvalidLoadOptionException;
-```
-
-```  
-/**
-* Build a {@link CarbonWriter}, which accepts Avro format object
-* @return AvroCarbonWriter 
-* @throws IOException
-* @throws InvalidLoadOptionException
-*/
-public CarbonWriter buildWriterForAvroInput() throws IOException, InvalidLoadOptionException;
-```
-
-### Class org.apache.carbondata.sdk.file.CarbonWriter
-```
-/**
-* Write an object to the file, the format of the object depends on the implementation
-* If AvroCarbonWriter, object is of type org.apache.avro.generic.GenericData.Record 
-* If CSVCarbonWriter, object is of type String[]
-* Note: This API is not thread safe
-* @param object
-* @throws IOException
-*/
-public abstract void write(Object object) throws IOException;
-```
-
-```
-/**
-* Flush and close the writer
-*/
-public abstract void close() throws IOException;
-```
-
-```
-/**
-* Create a {@link CarbonWriterBuilder} to build a {@link CarbonWriter}
-*/
-public static CarbonWriterBuilder builder() {
-return new CarbonWriterBuilder();
-}
-```
-
-### Class org.apache.carbondata.sdk.file.Field
-```
-/**
-* Field Constructor
-* @param name name of the field
-* @param type datatype of field, specified in strings.
-*/
-public Field(String name, String type);
-```
-
-```
-/**
-* Field constructor
-* @param name name of the field
-* @param type datatype of the field of class DataType
-*/
-public Field(String name, DataType type);  
-```
-
-### Class org.apache.carbondata.sdk.file.Schema
-
-```
-/**
-* construct a schema with fields
-* @param fields
-*/
-public Schema(Field[] fields);
-```
-
-```
-/**
-* Create a Schema using JSON string, for example:
-* [
-*   {"name":"string"},
-*   {"age":"int"}
-* ] 
-* @param json specified as string
-* @return Schema
-*/
-public static Schema parseJson(String json);
-```
-
-### Class org.apache.carbondata.sdk.file.AvroCarbonWriter
-```
-/**
-* converts avro schema to carbon schema, required by carbonWriter
-*
-* @param avroSchemaString json formatted avro schema as string
-* @return carbon sdk schema
-*/
-public static org.apache.carbondata.sdk.file.Schema getCarbonSchemaFromAvroSchema(String avroSchemaString);
-```
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/site/markdown/streaming-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/streaming-guide.md b/src/site/markdown/streaming-guide.md
index 3ea2881..a9b174f 100644
--- a/src/site/markdown/streaming-guide.md
+++ b/src/site/markdown/streaming-guide.md
@@ -27,11 +27,11 @@ Start spark-shell in new terminal, type :paste, then copy and run the following
  import org.apache.spark.sql.{CarbonEnv, SparkSession}
  import org.apache.spark.sql.CarbonSession._
  import org.apache.spark.sql.streaming.{ProcessingTime, StreamingQuery}
- import org.apache.carbondata.core.util.path.CarbonStorePath
- 
+ import org.apache.carbondata.core.util.path.CarbonTablePath
+
  val warehouse = new File("./warehouse").getCanonicalPath
  val metastore = new File("./metastore").getCanonicalPath
- 
+
  val spark = SparkSession
    .builder()
    .master("local")
@@ -54,8 +54,8 @@ Start spark-shell in new terminal, type :paste, then copy and run the following
       | TBLPROPERTIES('streaming'='true')""".stripMargin)
 
  val carbonTable = CarbonEnv.getCarbonTable(Some("default"), "carbon_table")(spark)
- val tablePath = CarbonStorePath.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
- 
+ val tablePath = carbonTable.getTablePath
+
  // batch load
  var qry: StreamingQuery = null
  val readSocketDF = spark.readStream
@@ -68,7 +68,7 @@ Start spark-shell in new terminal, type :paste, then copy and run the following
  qry = readSocketDF.writeStream
    .format("carbondata")
    .trigger(ProcessingTime("5 seconds"))
-   .option("checkpointLocation", tablePath.getStreamingCheckpointDir)
+   .option("checkpointLocation", CarbonTablePath.getStreamingCheckpointDir(tablePath))
    .option("dbName", "default")
    .option("tableName", "carbon_table")
    .start()

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/site/markdown/supported-data-types-in-carbondata.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/supported-data-types-in-carbondata.md b/src/site/markdown/supported-data-types-in-carbondata.md
index 6c21508..7260afe 100644
--- a/src/site/markdown/supported-data-types-in-carbondata.md
+++ b/src/site/markdown/supported-data-types-in-carbondata.md
@@ -38,6 +38,8 @@
   * Complex Types
     * arrays: ARRAY``<data_type>``
     * structs: STRUCT``<col_name : data_type COMMENT col_comment, ...>``
+    
+    **NOTE**: Only 2 level complex type schema is supported for now.
 
   * Other Types
     * BOOLEAN
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/site/markdown/timeseries-datamap-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/timeseries-datamap-guide.md b/src/site/markdown/timeseries-datamap-guide.md
index 7847312..bea5286 100644
--- a/src/site/markdown/timeseries-datamap-guide.md
+++ b/src/site/markdown/timeseries-datamap-guide.md
@@ -1,12 +1,12 @@
 # CarbonData Timeseries DataMap
 
-* [Timeseries DataMap](#timeseries-datamap-intoduction-(alpha-feature-in-1.3.0))
+* [Timeseries DataMap Introduction](#timeseries-datamap-intoduction)
 * [Compaction](#compacting-pre-aggregate-tables)
 * [Data Management](#data-management-with-pre-aggregate-tables)
 
-## Timeseries DataMap Intoduction (Alpha feature in 1.3.0)
-Timeseries DataMap a pre-aggregate table implementation based on 'preaggregate' DataMap. 
-Difference is that Timerseries DataMap has built-in understanding of time hierarchy and 
+## Timeseries DataMap Introduction (Alpha feature in 1.3.0)
+Timeseries DataMap a pre-aggregate table implementation based on 'pre-aggregate' DataMap.
+Difference is that Timeseries DataMap has built-in understanding of time hierarchy and
 levels: year, month, day, hour, minute, so that it supports automatic roll-up in time dimension 
 for query.
 

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/site/pdf.xml
----------------------------------------------------------------------
diff --git a/src/site/pdf.xml b/src/site/pdf.xml
index e8449a1..5fc3341 100644
--- a/src/site/pdf.xml
+++ b/src/site/pdf.xml
@@ -16,8 +16,10 @@
         <item name="Installation" ref='installation-guide.md'/>
         <item name="Configuring CarbonData" ref='configuration-parameters.md'/>
         <item name="Streaming Guide" ref='streaming-guide.md'/>
-      <item name="SDK Writer Guide" ref='sdk-writer-guide.md'/>
-      <item name="DataMap Developer Guide" ref='datamap-developer-guide.md'/>
+        <item name="SDK Guide" ref='sdk-guide.md'/>
+        <item name="DataMap Developer Guide" ref='datamap-developer-guide.md'/>
+        <item name="CarbonData BloomFilter DataMap (Alpha feature in 1.4.0)" ref='bloomfilter-datamap-guide.md'/>
+        <item name="CarbonData Lucene DataMap (Alpha feature in 1.4.0)" ref='lucene-datamap-guide.md'/>
         <item name="CarbonData Pre-aggregate DataMap" ref='preaggregate-datamap-guide.md'/>
         <item name="CarbonData Timeseries DataMap" ref='timeseries-datamap-guide.md'/>
         <item name="FAQs" ref='faq.md'/>
@@ -30,7 +32,7 @@
     <companyLogo>../../src/site/projectLogo/ApacheLogo.png</companyLogo>
     <projectLogo>../../src/site/projectLogo/CarbonDataLogo.png</projectLogo>
     <coverTitle>Apache CarbonData</coverTitle>
-    <coverSubTitle>Ver 1.3.1 </coverSubTitle>
+    <coverSubTitle>Ver 1.4.0 </coverSubTitle>
     <coverType>Documentation</coverType>
     <projectName>Apache CarbonData</projectName>
     <companyName>The Apache Software Foundation</companyName>


[3/5] carbondata-site git commit: doucment for latest release of carbondata 1.4.0

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

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/sdk-writer-guide.html
----------------------------------------------------------------------
diff --git a/content/sdk-writer-guide.html b/content/sdk-writer-guide.html
index a73c22c..36bb9ad 100644
--- a/content/sdk-writer-guide.html
+++ b/content/sdk-writer-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/security.html
----------------------------------------------------------------------
diff --git a/content/security.html b/content/security.html
index 826ed4a..2b0ab61 100644
--- a/content/security.html
+++ b/content/security.html
@@ -45,6 +45,9 @@
                            aria-expanded="false">Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/streaming-guide.html
----------------------------------------------------------------------
diff --git a/content/streaming-guide.html b/content/streaming-guide.html
index e9788a9..981a095 100644
--- a/content/streaming-guide.html
+++ b/content/streaming-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>
@@ -194,11 +197,11 @@
  <span class="pl-k">import</span> <span class="pl-en">org</span>.<span class="pl-en">apache</span>.<span class="pl-en">spark</span>.<span class="pl-en">sql</span>.{<span class="pl-en">CarbonEnv</span>, <span class="pl-en">SparkSession</span>}
  <span class="pl-k">import</span> <span class="pl-en">org</span>.<span class="pl-en">apache</span>.<span class="pl-en">spark</span>.<span class="pl-en">sql</span>.<span class="pl-en">CarbonSession</span>.<span class="pl-en">_</span>
  <span class="pl-k">import</span> <span class="pl-en">org</span>.<span class="pl-en">apache</span>.<span class="pl-en">spark</span>.<span class="pl-en">sql</span>.<span class="pl-en">streaming</span>.{<span class="pl-en">ProcessingTime</span>, <span class="pl-en">StreamingQuery</span>}
- <span class="pl-k">import</span> <span class="pl-en">org</span>.<span class="pl-en">apache</span>.<span class="pl-en">carbondata</span>.<span class="pl-en">core</span>.<span class="pl-en">util</span>.<span class="pl-en">path</span>.<span class="pl-en">CarbonStorePath</span>
- 
+ <span class="pl-k">import</span> <span class="pl-en">org</span>.<span class="pl-en">apache</span>.<span class="pl-en">carbondata</span>.<span class="pl-en">core</span>.<span class="pl-en">util</span>.<span class="pl-en">path</span>.<span class="pl-en">CarbonTablePath</span>
+
  <span class="pl-k">val</span> <span class="pl-smi">warehouse</span> <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-en">File</span>(<span class="pl-s"><span class="pl-pds">"</span>./warehouse<span class="pl-pds">"</span></span>).getCanonicalPath
  <span class="pl-k">val</span> <span class="pl-smi">metastore</span> <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-en">File</span>(<span class="pl-s"><span class="pl-pds">"</span>./metastore<span class="pl-pds">"</span></span>).getCanonicalPath
- 
+
  <span class="pl-k">val</span> <span class="pl-smi">spark</span> <span class="pl-k">=</span> <span class="pl-en">SparkSession</span>
    .builder()
    .master(<span class="pl-s"><span class="pl-pds">"</span>local<span class="pl-pds">"</span></span>)
@@ -221,8 +224,8 @@
 <span class="pl-s">      | TBLPROPERTIES('streaming'='true')<span class="pl-pds">"""</span></span>.stripMargin)
 
  <span class="pl-k">val</span> <span class="pl-smi">carbonTable</span> <span class="pl-k">=</span> <span class="pl-en">CarbonEnv</span>.getCarbonTable(<span class="pl-en">Some</span>(<span class="pl-s"><span class="pl-pds">"</span>default<span class="pl-pds">"</span></span>), <span class="pl-s"><span class="pl-pds">"</span>carbon_table<span class="pl-pds">"</span></span>)(spark)
- <span class="pl-k">val</span> <span class="pl-smi">tablePath</span> <span class="pl-k">=</span> <span class="pl-en">CarbonStorePath</span>.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
- 
+ <span class="pl-k">val</span> <span class="pl-smi">tablePath</span> <span class="pl-k">=</span> carbonTable.getTablePath
+
  <span class="pl-c"><span class="pl-c">//</span> batch load</span>
  <span class="pl-k">var</span> <span class="pl-smi">qry</span><span class="pl-k">:</span> <span class="pl-en">StreamingQuery</span> <span class="pl-k">=</span> <span class="pl-c1">null</span>
  <span class="pl-k">val</span> <span class="pl-smi">readSocketDF</span> <span class="pl-k">=</span> spark.readStream
@@ -235,7 +238,7 @@
  qry <span class="pl-k">=</span> readSocketDF.writeStream
    .format(<span class="pl-s"><span class="pl-pds">"</span>carbondata<span class="pl-pds">"</span></span>)
    .trigger(<span class="pl-en">ProcessingTime</span>(<span class="pl-s"><span class="pl-pds">"</span>5 seconds<span class="pl-pds">"</span></span>))
-   .option(<span class="pl-s"><span class="pl-pds">"</span>checkpointLocation<span class="pl-pds">"</span></span>, tablePath.getStreamingCheckpointDir)
+   .option(<span class="pl-s"><span class="pl-pds">"</span>checkpointLocation<span class="pl-pds">"</span></span>, <span class="pl-en">CarbonTablePath</span>.getStreamingCheckpointDir(tablePath))
    .option(<span class="pl-s"><span class="pl-pds">"</span>dbName<span class="pl-pds">"</span></span>, <span class="pl-s"><span class="pl-pds">"</span>default<span class="pl-pds">"</span></span>)
    .option(<span class="pl-s"><span class="pl-pds">"</span>tableName<span class="pl-pds">"</span></span>, <span class="pl-s"><span class="pl-pds">"</span>carbon_table<span class="pl-pds">"</span></span>)
    .start()

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/supported-data-types-in-carbondata.html
----------------------------------------------------------------------
diff --git a/content/supported-data-types-in-carbondata.html b/content/supported-data-types-in-carbondata.html
index 43d1071..0f06a42 100644
--- a/content/supported-data-types-in-carbondata.html
+++ b/content/supported-data-types-in-carbondata.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>
@@ -212,6 +215,7 @@
 <li>structs: STRUCT<code>&lt;col_name : data_type COMMENT col_comment, ...&gt;</code>
 </li>
 </ul>
+<p><strong>NOTE</strong>: Only 2 level complex type schema is supported for now.</p>
 </li>
 <li>
 <p>Other Types</p>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/timeseries-datamap-guide.html
----------------------------------------------------------------------
diff --git a/content/timeseries-datamap-guide.html b/content/timeseries-datamap-guide.html
index 300e879..b40150c 100644
--- a/content/timeseries-datamap-guide.html
+++ b/content/timeseries-datamap-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/troubleshooting.html
----------------------------------------------------------------------
diff --git a/content/troubleshooting.html b/content/troubleshooting.html
index 37a07b4..d5cf1b2 100644
--- a/content/troubleshooting.html
+++ b/content/troubleshooting.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/useful-tips-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/useful-tips-on-carbondata.html b/content/useful-tips-on-carbondata.html
index 8b653b4..e8b0ab0 100644
--- a/content/useful-tips-on-carbondata.html
+++ b/content/useful-tips-on-carbondata.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/videogallery.html
----------------------------------------------------------------------
diff --git a/content/videogallery.html b/content/videogallery.html
index 2dd8c67..3c9e5eb 100644
--- a/content/videogallery.html
+++ b/content/videogallery.html
@@ -49,6 +49,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/resources/application.conf
----------------------------------------------------------------------
diff --git a/src/main/resources/application.conf b/src/main/resources/application.conf
index ef8c3dc..df5280b 100644
--- a/src/main/resources/application.conf
+++ b/src/main/resources/application.conf
@@ -8,10 +8,12 @@ fileList=["configuration-parameters",
   "supported-data-types-in-carbondata",
   "troubleshooting",
   "useful-tips-on-carbondata",
-  "sdk-writer-guide",
+  "sdk-guide",
   "datamap-developer-guide"
   ]
 dataMapFileList=[
+  "bloomfilter-datamap-guide",
+  "lucene-datamap-guide",
   "preaggregate-datamap-guide",
   "timeseries-datamap-guide"]
 

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/scala/html/header.html
----------------------------------------------------------------------
diff --git a/src/main/scala/html/header.html b/src/main/scala/html/header.html
index d2ebd58..470de01 100644
--- a/src/main/scala/html/header.html
+++ b/src/main/scala/html/header.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/bloomfilter-datamap-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/bloomfilter-datamap-guide.html b/src/main/webapp/bloomfilter-datamap-guide.html
new file mode 100644
index 0000000..41aef5c
--- /dev/null
+++ b/src/main/webapp/bloomfilter-datamap-guide.html
@@ -0,0 +1,306 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
+                                   target="_blank">Apache CarbonData 1.2.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
+                                   target="_blank">Apache CarbonData 1.1.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
+                                   target="_blank">Apache CarbonData 1.1.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
+                                   target="_blank">Apache CarbonData 1.0.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.2.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.1</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="row">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div><h1>
+<a id="carbondata-bloomfilter-datamap-alpha-feature-in-140" class="anchor" href="#carbondata-bloomfilter-datamap-alpha-feature-in-140" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData BloomFilter DataMap (Alpha feature in 1.4.0)</h1>
+<ul>
+<li><a href="#datamap-management">DataMap Management</a></li>
+<li><a href="#bloomfilter-datamap-introduction">BloomFilter Datamap Introduction</a></li>
+<li><a href="#loading-data">Loading Data</a></li>
+<li><a href="#querying-data">Querying Data</a></li>
+<li><a href="#data-management-with-bloomfilter-datamap">Data Management</a></li>
+</ul>
+<h4>
+<a id="datamap-management" class="anchor" href="#datamap-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DataMap Management</h4>
+<p>Creating BloomFilter DataMap</p>
+<pre><code>CREATE DATAMAP [IF NOT EXISTS] datamap_name
+ON TABLE main_table
+USING 'bloomfilter'
+DMPROPERTIES ('index_columns'='city, name', 'BLOOM_SIZE'='640000', 'BLOOM_FPP'='0.00001')
+</code></pre>
+<p>Dropping specified datamap</p>
+<pre><code>DROP DATAMAP [IF EXISTS] datamap_name
+ON TABLE main_table
+</code></pre>
+<p>Showing all DataMaps on this table</p>
+<pre><code>SHOW DATAMAP
+ON TABLE main_table
+</code></pre>
+<p>It will show all DataMaps created on main table.</p>
+<h2>
+<a id="bloomfilter-datamap-introduction" class="anchor" href="#bloomfilter-datamap-introduction" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>BloomFilter DataMap Introduction</h2>
+<p>A Bloom filter is a space-efficient probabilistic data structure that is used to test whether an element is a member of a set.
+Carbondata introduce BloomFilter as an index datamap to enhance the performance of querying with precise value.
+It is well suitable for queries that do precise match on high cardinality columns(such as Name/ID).
+Internally, CarbonData maintains a BloomFilter per blocklet for each index column to indicate that whether a value of the column is in this blocklet.
+Just like the other datamaps, BloomFilter datamap is managed ablong with main tables by CarbonData.
+User can create BloomFilter datamap on specified columns with specified BloomFilter configurations such as size and probability.</p>
+<p>For instance, main table called <strong>datamap_test</strong> which is defined as:</p>
+<pre><code>CREATE TABLE datamap_test (
+  id string,
+  name string,
+  age int,
+  city string,
+  country string)
+STORED BY 'carbondata'
+TBLPROPERTIES('SORT_COLUMNS'='id')
+</code></pre>
+<p>In the above example, <code>id</code> and <code>name</code> are high cardinality columns
+and we always query on <code>id</code> and <code>name</code> with precise value.
+since <code>id</code> is in the sort_columns and it is orderd,
+query on it will be fast because CarbonData can skip all the irrelative blocklets.
+But queries on <code>name</code> may be bad since the blocklet minmax may not help,
+because in each blocklet the range of the value of <code>name</code> may be the same -- all from A<em>~z</em>.
+In this case, user can create a BloomFilter datamap on column <code>name</code>.
+Moreover, user can also create a BloomFilter datamap on the sort_columns.
+This is useful if user has too many segments and the range of the value of sort_columns are almost the same.</p>
+<p>User can create BloomFilter datamap using the Create DataMap DDL:</p>
+<pre><code>CREATE DATAMAP dm
+ON TABLE datamap_test
+USING 'bloomfilter'
+DMPROPERTIES ('INDEX_COLUMNS' = 'name,id', 'BLOOM_SIZE'='640000', 'BLOOM_FPP'='0.00001', 'BLOOM_COMPRESS'='true')
+</code></pre>
+<p><strong>Properties for BloomFilter DataMap</strong></p>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Is Required</th>
+<th>Default Value</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>INDEX_COLUMNS</td>
+<td>YES</td>
+<td></td>
+<td>Carbondata will generate BloomFilter index on these columns. Queries on there columns are usually like 'COL = VAL'.</td>
+</tr>
+<tr>
+<td>BLOOM_SIZE</td>
+<td>NO</td>
+<td>32000</td>
+<td>This value is internally used by BloomFilter as the number of expected insertions, it will affects the size of BloomFilter index. Since each blocklet has a BloomFilter here, so the value is the approximate records in a blocklet. In another word, the value 32000 * #noOfPagesInBlocklet. The value should be an integer.</td>
+</tr>
+<tr>
+<td>BLOOM_FPP</td>
+<td>NO</td>
+<td>0.01</td>
+<td>This value is internally used by BloomFilter as the False-Positive Probability, it will affects the size of bloomfilter index as well as the number of hash functions for the BloomFilter. The value should be in range (0, 1).</td>
+</tr>
+<tr>
+<td>BLOOM_COMPRESS</td>
+<td>NO</td>
+<td>true</td>
+<td>Whether to compress the BloomFilter index files.</td>
+</tr>
+</tbody>
+</table>
+<h2>
+<a id="loading-data" class="anchor" href="#loading-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Loading Data</h2>
+<p>When loading data to main table, BloomFilter files will be generated for all the
+index_columns given in DMProperties which contains the blockletId and a BloomFilter for each index column.
+These index files will be written inside a folder named with datamap name
+inside each segment folders.</p>
+<h2>
+<a id="querying-data" class="anchor" href="#querying-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Querying Data</h2>
+<p>A system level configuration <code>carbon.query.datamap.bloom.cache.size</code> can used to enhance query performance with BloomFilter datamap by providing a cache for the bloomfilter index files.
+The default value is <code>512</code> and its unit is <code>MB</code>. Internally the cache will be expired after it's idle for 2 hours.</p>
+<p>User can verify whether a query can leverage BloomFilter datamap by executing <code>EXPLAIN</code> command,
+which will show the transformed logical plan, and thus user can check whether the BloomFilter datamap can skip blocklets during the scan.
+If the datamap does not prune blocklets well, you can try to increase the value of property <code>BLOOM_SIZE</code> and decrease the value of property <code>BLOOM_FPP</code>.</p>
+<h2>
+<a id="data-management-with-bloomfilter-datamap" class="anchor" href="#data-management-with-bloomfilter-datamap" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Management With BloomFilter DataMap</h2>
+<p>Data management with BloomFilter datamap has no difference with that on Lucene datamap. You can refer to the corresponding section in <code>CarbonData BloomFilter DataMap</code>.</p>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/configuration-parameters.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/configuration-parameters.html b/src/main/webapp/configuration-parameters.html
index 8db28db..2768b0e 100644
--- a/src/main/webapp/configuration-parameters.html
+++ b/src/main/webapp/configuration-parameters.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>
@@ -590,6 +593,11 @@
 <td>0.8</td>
 <td>Specifies the minimum resource (executor) ratio needed for starting the block distribution. The default value is 0.8, which indicates 80% of the requested resource is allocated for starting block distribution.  The minimum value is 0.1 min and the maximum value is 1.0.</td>
 </tr>
+<tr>
+<td>carbon.search.enabled</td>
+<td>false</td>
+<td>If set to true, it will use CarbonReader to do distributed scan directly instead of using compute framework like spark, thus avoiding limitation of compute framework like SQL optimizer and task scheduling overhead.</td>
+</tr>
 </tbody>
 </table>
 <ul>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/data-management-on-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/data-management-on-carbondata.html b/src/main/webapp/data-management-on-carbondata.html
index 49f2fe8..a9c7926 100644
--- a/src/main/webapp/data-management-on-carbondata.html
+++ b/src/main/webapp/data-management-on-carbondata.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/datamap-developer-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/datamap-developer-guide.html b/src/main/webapp/datamap-developer-guide.html
index 18ec74d..367fcb7 100644
--- a/src/main/webapp/datamap-developer-guide.html
+++ b/src/main/webapp/datamap-developer-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/faq.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/faq.html b/src/main/webapp/faq.html
index 517fea2..5d43ff1 100644
--- a/src/main/webapp/faq.html
+++ b/src/main/webapp/faq.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/file-structure-of-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/file-structure-of-carbondata.html b/src/main/webapp/file-structure-of-carbondata.html
index 5ea9cbb..dc1e239 100644
--- a/src/main/webapp/file-structure-of-carbondata.html
+++ b/src/main/webapp/file-structure-of-carbondata.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/index.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/index.html b/src/main/webapp/index.html
index 3316f63..3fe3ba4 100644
--- a/src/main/webapp/index.html
+++ b/src/main/webapp/index.html
@@ -54,6 +54,9 @@
                                 class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>
@@ -325,6 +328,13 @@
                                 </h4>
                                 <div class="linkblock">
                                     <div class="block-row">
+                                        <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                           target="_blank">Apache CarbonData 1.4.0</a>
+                                        <span class="release-date">May 2018</span>
+                                        <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Apache+CarbonData+1.4.0+Release"
+                                           class="whatsnew" target="_blank">what's new</a>
+                                    </div>
+                                    <div class="block-row">
                                         <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                            target="_blank">Apache CarbonData 1.3.1</a>
                                         <span class="release-date">Mar 2018</span>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/installation-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/installation-guide.html b/src/main/webapp/installation-guide.html
index 6d944f7..ffb750e 100644
--- a/src/main/webapp/installation-guide.html
+++ b/src/main/webapp/installation-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>


[5/5] carbondata-site git commit: add release 1.4.0

Posted by ch...@apache.org.
add release 1.4.0


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

Branch: refs/heads/asf-site
Commit: 61436c8e415519d04b6fea6ebfaf62bc1122699c
Parents: 8cb979c 2941aac
Author: chenliang613 <ch...@huawei.com>
Authored: Sat Jun 2 11:25:57 2018 +0800
Committer: chenliang613 <ch...@huawei.com>
Committed: Sat Jun 2 11:25:57 2018 +0800

----------------------------------------------------------------------
 content/WEB-INF/classes/application.conf        |   4 +-
 content/bloomfilter-datamap-guide.html          | 306 +++++++
 content/configuration-parameters.html           |   8 +
 content/data-management-on-carbondata.html      |   3 +
 content/datamap-developer-guide.html            |   3 +
 content/faq.html                                |   3 +
 content/file-structure-of-carbondata.html       |   3 +
 content/index.html                              |   3 +
 content/installation-guide.html                 |   3 +
 content/lucene-datamap-guide.html               | 337 ++++++++
 content/mainpage.html                           |  16 +-
 content/pdf/CarbonData Documentation.pdf        | Bin 264034 -> 292327 bytes
 content/preaggregate-datamap-guide.html         |   3 +
 content/quick-start-guide.html                  |   3 +
 content/sdk-guide.html                          | 820 +++++++++++++++++++
 content/sdk-writer-guide.html                   |   3 +
 content/security.html                           |   3 +
 content/streaming-guide.html                    |  15 +-
 content/supported-data-types-in-carbondata.html |   4 +
 content/timeseries-datamap-guide.html           |   3 +
 content/troubleshooting.html                    |   3 +
 content/useful-tips-on-carbondata.html          |   3 +
 content/videogallery.html                       |   3 +
 src/main/resources/application.conf             |   4 +-
 src/main/scala/html/header.html                 |   3 +
 src/main/webapp/bloomfilter-datamap-guide.html  | 306 +++++++
 src/main/webapp/configuration-parameters.html   |   8 +
 .../webapp/data-management-on-carbondata.html   |   3 +
 src/main/webapp/datamap-developer-guide.html    |   3 +
 src/main/webapp/faq.html                        |   3 +
 .../webapp/file-structure-of-carbondata.html    |   3 +
 src/main/webapp/index.html                      |  10 +
 src/main/webapp/installation-guide.html         |   3 +
 src/main/webapp/lucene-datamap-guide.html       | 337 ++++++++
 src/main/webapp/mainpage.html                   |  16 +-
 .../webapp/pdf/CarbonData Documentation.pdf     | Bin 264034 -> 292327 bytes
 src/main/webapp/preaggregate-datamap-guide.html |   3 +
 src/main/webapp/quick-start-guide.html          |   3 +
 src/main/webapp/sdk-guide.html                  | 820 +++++++++++++++++++
 src/main/webapp/sdk-writer-guide.html           |   3 +
 src/main/webapp/security.html                   |   3 +
 src/main/webapp/streaming-guide.html            |  15 +-
 .../supported-data-types-in-carbondata.html     |   4 +
 src/main/webapp/timeseries-datamap-guide.html   |   3 +
 src/main/webapp/troubleshooting.html            |   3 +
 src/main/webapp/useful-tips-on-carbondata.html  |   3 +
 src/main/webapp/videogallery.html               |   3 +
 src/site/markdown/bloomfilter-datamap-guide.md  |  98 +++
 src/site/markdown/configuration-parameters.md   |   3 +-
 .../markdown/data-management-on-carbondata.md   |   4 +-
 src/site/markdown/lucene-datamap-guide.md       | 159 ++++
 src/site/markdown/sdk-guide.md                  | 680 +++++++++++++++
 src/site/markdown/sdk-writer-guide.md           | 359 --------
 src/site/markdown/streaming-guide.md            |  12 +-
 .../supported-data-types-in-carbondata.md       |   2 +
 src/site/markdown/timeseries-datamap-guide.md   |   8 +-
 src/site/pdf.xml                                |   8 +-
 57 files changed, 4044 insertions(+), 403 deletions(-)
----------------------------------------------------------------------



[4/5] carbondata-site git commit: doucment for latest release of carbondata 1.4.0

Posted by ch...@apache.org.
doucment for latest release of carbondata 1.4.0


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

Branch: refs/heads/asf-site
Commit: 2941aacaca5b8278416dfbea8e1be4b27db33f5a
Parents: 8cb979c
Author: vandana7 <va...@gmail.com>
Authored: Wed May 30 18:38:20 2018 +0530
Committer: vandana7 <va...@gmail.com>
Committed: Wed May 30 18:38:20 2018 +0530

----------------------------------------------------------------------
 content/WEB-INF/classes/application.conf        |   4 +-
 content/bloomfilter-datamap-guide.html          | 306 +++++++
 content/configuration-parameters.html           |   8 +
 content/data-management-on-carbondata.html      |   3 +
 content/datamap-developer-guide.html            |   3 +
 content/faq.html                                |   3 +
 content/file-structure-of-carbondata.html       |   3 +
 content/index.html                              |   3 +
 content/installation-guide.html                 |   3 +
 content/lucene-datamap-guide.html               | 337 ++++++++
 content/mainpage.html                           |  16 +-
 content/pdf/CarbonData Documentation.pdf        | Bin 264034 -> 292327 bytes
 content/preaggregate-datamap-guide.html         |   3 +
 content/quick-start-guide.html                  |   3 +
 content/sdk-guide.html                          | 820 +++++++++++++++++++
 content/sdk-writer-guide.html                   |   3 +
 content/security.html                           |   3 +
 content/streaming-guide.html                    |  15 +-
 content/supported-data-types-in-carbondata.html |   4 +
 content/timeseries-datamap-guide.html           |   3 +
 content/troubleshooting.html                    |   3 +
 content/useful-tips-on-carbondata.html          |   3 +
 content/videogallery.html                       |   3 +
 src/main/resources/application.conf             |   4 +-
 src/main/scala/html/header.html                 |   3 +
 src/main/webapp/bloomfilter-datamap-guide.html  | 306 +++++++
 src/main/webapp/configuration-parameters.html   |   8 +
 .../webapp/data-management-on-carbondata.html   |   3 +
 src/main/webapp/datamap-developer-guide.html    |   3 +
 src/main/webapp/faq.html                        |   3 +
 .../webapp/file-structure-of-carbondata.html    |   3 +
 src/main/webapp/index.html                      |  10 +
 src/main/webapp/installation-guide.html         |   3 +
 src/main/webapp/lucene-datamap-guide.html       | 337 ++++++++
 src/main/webapp/mainpage.html                   |  16 +-
 .../webapp/pdf/CarbonData Documentation.pdf     | Bin 264034 -> 292327 bytes
 src/main/webapp/preaggregate-datamap-guide.html |   3 +
 src/main/webapp/quick-start-guide.html          |   3 +
 src/main/webapp/sdk-guide.html                  | 820 +++++++++++++++++++
 src/main/webapp/sdk-writer-guide.html           |   3 +
 src/main/webapp/security.html                   |   3 +
 src/main/webapp/streaming-guide.html            |  15 +-
 .../supported-data-types-in-carbondata.html     |   4 +
 src/main/webapp/timeseries-datamap-guide.html   |   3 +
 src/main/webapp/troubleshooting.html            |   3 +
 src/main/webapp/useful-tips-on-carbondata.html  |   3 +
 src/main/webapp/videogallery.html               |   3 +
 src/site/markdown/bloomfilter-datamap-guide.md  |  98 +++
 src/site/markdown/configuration-parameters.md   |   3 +-
 .../markdown/data-management-on-carbondata.md   |   4 +-
 src/site/markdown/lucene-datamap-guide.md       | 159 ++++
 src/site/markdown/sdk-guide.md                  | 680 +++++++++++++++
 src/site/markdown/sdk-writer-guide.md           | 359 --------
 src/site/markdown/streaming-guide.md            |  12 +-
 .../supported-data-types-in-carbondata.md       |   2 +
 src/site/markdown/timeseries-datamap-guide.md   |   8 +-
 src/site/pdf.xml                                |   8 +-
 57 files changed, 4044 insertions(+), 403 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/WEB-INF/classes/application.conf
----------------------------------------------------------------------
diff --git a/content/WEB-INF/classes/application.conf b/content/WEB-INF/classes/application.conf
index ef8c3dc..df5280b 100644
--- a/content/WEB-INF/classes/application.conf
+++ b/content/WEB-INF/classes/application.conf
@@ -8,10 +8,12 @@ fileList=["configuration-parameters",
   "supported-data-types-in-carbondata",
   "troubleshooting",
   "useful-tips-on-carbondata",
-  "sdk-writer-guide",
+  "sdk-guide",
   "datamap-developer-guide"
   ]
 dataMapFileList=[
+  "bloomfilter-datamap-guide",
+  "lucene-datamap-guide",
   "preaggregate-datamap-guide",
   "timeseries-datamap-guide"]
 

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/bloomfilter-datamap-guide.html
----------------------------------------------------------------------
diff --git a/content/bloomfilter-datamap-guide.html b/content/bloomfilter-datamap-guide.html
new file mode 100644
index 0000000..41aef5c
--- /dev/null
+++ b/content/bloomfilter-datamap-guide.html
@@ -0,0 +1,306 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
+                                   target="_blank">Apache CarbonData 1.2.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
+                                   target="_blank">Apache CarbonData 1.1.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
+                                   target="_blank">Apache CarbonData 1.1.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
+                                   target="_blank">Apache CarbonData 1.0.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.2.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.1</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="row">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div><h1>
+<a id="carbondata-bloomfilter-datamap-alpha-feature-in-140" class="anchor" href="#carbondata-bloomfilter-datamap-alpha-feature-in-140" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData BloomFilter DataMap (Alpha feature in 1.4.0)</h1>
+<ul>
+<li><a href="#datamap-management">DataMap Management</a></li>
+<li><a href="#bloomfilter-datamap-introduction">BloomFilter Datamap Introduction</a></li>
+<li><a href="#loading-data">Loading Data</a></li>
+<li><a href="#querying-data">Querying Data</a></li>
+<li><a href="#data-management-with-bloomfilter-datamap">Data Management</a></li>
+</ul>
+<h4>
+<a id="datamap-management" class="anchor" href="#datamap-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DataMap Management</h4>
+<p>Creating BloomFilter DataMap</p>
+<pre><code>CREATE DATAMAP [IF NOT EXISTS] datamap_name
+ON TABLE main_table
+USING 'bloomfilter'
+DMPROPERTIES ('index_columns'='city, name', 'BLOOM_SIZE'='640000', 'BLOOM_FPP'='0.00001')
+</code></pre>
+<p>Dropping specified datamap</p>
+<pre><code>DROP DATAMAP [IF EXISTS] datamap_name
+ON TABLE main_table
+</code></pre>
+<p>Showing all DataMaps on this table</p>
+<pre><code>SHOW DATAMAP
+ON TABLE main_table
+</code></pre>
+<p>It will show all DataMaps created on main table.</p>
+<h2>
+<a id="bloomfilter-datamap-introduction" class="anchor" href="#bloomfilter-datamap-introduction" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>BloomFilter DataMap Introduction</h2>
+<p>A Bloom filter is a space-efficient probabilistic data structure that is used to test whether an element is a member of a set.
+Carbondata introduce BloomFilter as an index datamap to enhance the performance of querying with precise value.
+It is well suitable for queries that do precise match on high cardinality columns(such as Name/ID).
+Internally, CarbonData maintains a BloomFilter per blocklet for each index column to indicate that whether a value of the column is in this blocklet.
+Just like the other datamaps, BloomFilter datamap is managed ablong with main tables by CarbonData.
+User can create BloomFilter datamap on specified columns with specified BloomFilter configurations such as size and probability.</p>
+<p>For instance, main table called <strong>datamap_test</strong> which is defined as:</p>
+<pre><code>CREATE TABLE datamap_test (
+  id string,
+  name string,
+  age int,
+  city string,
+  country string)
+STORED BY 'carbondata'
+TBLPROPERTIES('SORT_COLUMNS'='id')
+</code></pre>
+<p>In the above example, <code>id</code> and <code>name</code> are high cardinality columns
+and we always query on <code>id</code> and <code>name</code> with precise value.
+since <code>id</code> is in the sort_columns and it is orderd,
+query on it will be fast because CarbonData can skip all the irrelative blocklets.
+But queries on <code>name</code> may be bad since the blocklet minmax may not help,
+because in each blocklet the range of the value of <code>name</code> may be the same -- all from A<em>~z</em>.
+In this case, user can create a BloomFilter datamap on column <code>name</code>.
+Moreover, user can also create a BloomFilter datamap on the sort_columns.
+This is useful if user has too many segments and the range of the value of sort_columns are almost the same.</p>
+<p>User can create BloomFilter datamap using the Create DataMap DDL:</p>
+<pre><code>CREATE DATAMAP dm
+ON TABLE datamap_test
+USING 'bloomfilter'
+DMPROPERTIES ('INDEX_COLUMNS' = 'name,id', 'BLOOM_SIZE'='640000', 'BLOOM_FPP'='0.00001', 'BLOOM_COMPRESS'='true')
+</code></pre>
+<p><strong>Properties for BloomFilter DataMap</strong></p>
+<table>
+<thead>
+<tr>
+<th>Property</th>
+<th>Is Required</th>
+<th>Default Value</th>
+<th>Description</th>
+</tr>
+</thead>
+<tbody>
+<tr>
+<td>INDEX_COLUMNS</td>
+<td>YES</td>
+<td></td>
+<td>Carbondata will generate BloomFilter index on these columns. Queries on there columns are usually like 'COL = VAL'.</td>
+</tr>
+<tr>
+<td>BLOOM_SIZE</td>
+<td>NO</td>
+<td>32000</td>
+<td>This value is internally used by BloomFilter as the number of expected insertions, it will affects the size of BloomFilter index. Since each blocklet has a BloomFilter here, so the value is the approximate records in a blocklet. In another word, the value 32000 * #noOfPagesInBlocklet. The value should be an integer.</td>
+</tr>
+<tr>
+<td>BLOOM_FPP</td>
+<td>NO</td>
+<td>0.01</td>
+<td>This value is internally used by BloomFilter as the False-Positive Probability, it will affects the size of bloomfilter index as well as the number of hash functions for the BloomFilter. The value should be in range (0, 1).</td>
+</tr>
+<tr>
+<td>BLOOM_COMPRESS</td>
+<td>NO</td>
+<td>true</td>
+<td>Whether to compress the BloomFilter index files.</td>
+</tr>
+</tbody>
+</table>
+<h2>
+<a id="loading-data" class="anchor" href="#loading-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Loading Data</h2>
+<p>When loading data to main table, BloomFilter files will be generated for all the
+index_columns given in DMProperties which contains the blockletId and a BloomFilter for each index column.
+These index files will be written inside a folder named with datamap name
+inside each segment folders.</p>
+<h2>
+<a id="querying-data" class="anchor" href="#querying-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Querying Data</h2>
+<p>A system level configuration <code>carbon.query.datamap.bloom.cache.size</code> can used to enhance query performance with BloomFilter datamap by providing a cache for the bloomfilter index files.
+The default value is <code>512</code> and its unit is <code>MB</code>. Internally the cache will be expired after it's idle for 2 hours.</p>
+<p>User can verify whether a query can leverage BloomFilter datamap by executing <code>EXPLAIN</code> command,
+which will show the transformed logical plan, and thus user can check whether the BloomFilter datamap can skip blocklets during the scan.
+If the datamap does not prune blocklets well, you can try to increase the value of property <code>BLOOM_SIZE</code> and decrease the value of property <code>BLOOM_FPP</code>.</p>
+<h2>
+<a id="data-management-with-bloomfilter-datamap" class="anchor" href="#data-management-with-bloomfilter-datamap" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Management With BloomFilter DataMap</h2>
+<p>Data management with BloomFilter datamap has no difference with that on Lucene datamap. You can refer to the corresponding section in <code>CarbonData BloomFilter DataMap</code>.</p>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/configuration-parameters.html
----------------------------------------------------------------------
diff --git a/content/configuration-parameters.html b/content/configuration-parameters.html
index 8db28db..2768b0e 100644
--- a/content/configuration-parameters.html
+++ b/content/configuration-parameters.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>
@@ -590,6 +593,11 @@
 <td>0.8</td>
 <td>Specifies the minimum resource (executor) ratio needed for starting the block distribution. The default value is 0.8, which indicates 80% of the requested resource is allocated for starting block distribution.  The minimum value is 0.1 min and the maximum value is 1.0.</td>
 </tr>
+<tr>
+<td>carbon.search.enabled</td>
+<td>false</td>
+<td>If set to true, it will use CarbonReader to do distributed scan directly instead of using compute framework like spark, thus avoiding limitation of compute framework like SQL optimizer and task scheduling overhead.</td>
+</tr>
 </tbody>
 </table>
 <ul>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/data-management-on-carbondata.html
----------------------------------------------------------------------
diff --git a/content/data-management-on-carbondata.html b/content/data-management-on-carbondata.html
index 49f2fe8..a9c7926 100644
--- a/content/data-management-on-carbondata.html
+++ b/content/data-management-on-carbondata.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/datamap-developer-guide.html
----------------------------------------------------------------------
diff --git a/content/datamap-developer-guide.html b/content/datamap-developer-guide.html
index 18ec74d..367fcb7 100644
--- a/content/datamap-developer-guide.html
+++ b/content/datamap-developer-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/faq.html
----------------------------------------------------------------------
diff --git a/content/faq.html b/content/faq.html
index 517fea2..5d43ff1 100644
--- a/content/faq.html
+++ b/content/faq.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/file-structure-of-carbondata.html
----------------------------------------------------------------------
diff --git a/content/file-structure-of-carbondata.html b/content/file-structure-of-carbondata.html
index 5ea9cbb..dc1e239 100644
--- a/content/file-structure-of-carbondata.html
+++ b/content/file-structure-of-carbondata.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/index.html
----------------------------------------------------------------------
diff --git a/content/index.html b/content/index.html
index 3316f63..c201fb2 100644
--- a/content/index.html
+++ b/content/index.html
@@ -54,6 +54,9 @@
                                 class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/installation-guide.html
----------------------------------------------------------------------
diff --git a/content/installation-guide.html b/content/installation-guide.html
index 6d944f7..ffb750e 100644
--- a/content/installation-guide.html
+++ b/content/installation-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/lucene-datamap-guide.html
----------------------------------------------------------------------
diff --git a/content/lucene-datamap-guide.html b/content/lucene-datamap-guide.html
new file mode 100644
index 0000000..5f56429
--- /dev/null
+++ b/content/lucene-datamap-guide.html
@@ -0,0 +1,337 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
+                                   target="_blank">Apache CarbonData 1.2.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
+                                   target="_blank">Apache CarbonData 1.1.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
+                                   target="_blank">Apache CarbonData 1.1.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
+                                   target="_blank">Apache CarbonData 1.0.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.2.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.1</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="row">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div><h1>
+<a id="carbondata-lucene-datamap-alpha-feature-in-140" class="anchor" href="#carbondata-lucene-datamap-alpha-feature-in-140" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Lucene DataMap (Alpha feature in 1.4.0)</h1>
+<ul>
+<li><a href="#datamap-management">DataMap Management</a></li>
+<li><a href="#lucene-datamap-introduction">Lucene Datamap</a></li>
+<li><a href="#loading-data">Loading Data</a></li>
+<li><a href="#querying-data">Querying Data</a></li>
+<li><a href="#data-management-with-lucene-datamap">Data Management</a></li>
+</ul>
+<h4>
+<a id="datamap-management" class="anchor" href="#datamap-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DataMap Management</h4>
+<p>Lucene DataMap can be created using following DDL</p>
+<pre><code>CREATE DATAMAP [IF NOT EXISTS] datamap_name
+ON TABLE main_table
+USING 'lucene'
+DMPROPERTIES ('index_columns'='city, name', ...)
+</code></pre>
+<p>DataMap can be dropped using following DDL:</p>
+<pre><code>DROP DATAMAP [IF EXISTS] datamap_name
+ON TABLE main_table
+</code></pre>
+<p>To show all DataMaps created, use:</p>
+<pre><code>SHOW DATAMAP 
+ON TABLE main_table
+</code></pre>
+<p>It will show all DataMaps created on main table.</p>
+<h2>
+<a id="lucene-datamap-introduction" class="anchor" href="#lucene-datamap-introduction" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Lucene DataMap Introduction</h2>
+<p>Lucene is a high performance, full featured text search engine. Lucene is integrated to carbon as
+an index datamap and managed along with main tables by CarbonData.User can create lucene datamap
+to improve query performance on string columns which has content of more length. So, user can
+search tokenized word or pattern of it using lucene query on text content.</p>
+<p>For instance, main table called <strong>datamap_test</strong> which is defined as:</p>
+<pre><code>CREATE TABLE datamap_test (
+  name string,
+  age int,
+  city string,
+  country string)
+STORED BY 'carbondata'
+</code></pre>
+<p>User can create Lucene datamap using the Create DataMap DDL:</p>
+<pre><code>CREATE DATAMAP dm
+ON TABLE datamap_test
+USING 'lucene'
+DMPROPERTIES ('INDEX_COLUMNS' = 'name, country',)
+</code></pre>
+<p><strong>DMProperties</strong></p>
+<ol>
+<li>INDEX_COLUMNS: The list of string columns on which lucene creates indexes.</li>
+<li>FLUSH_CACHE: size of the cache to maintain in Lucene writer, if specified then it tries to
+aggregate the unique data till the cache limit and flush to Lucene. It is best suitable for low
+cardinality dimensions.</li>
+<li>SPLIT_BLOCKLET: when made as true then store the data in blocklet wise in lucene , it means new
+folder will be created for each blocklet, thus, it eliminates storing blockletid in lucene and
+also it makes lucene small chunks of data.</li>
+</ol>
+<h2>
+<a id="loading-data" class="anchor" href="#loading-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Loading data</h2>
+<p>When loading data to main table, lucene index files will be generated for all the
+index_columns(String Columns) given in DMProperties which contains information about the data
+location of index_columns. These index files will be written inside a folder named with datamap name
+inside each segment folders.</p>
+<p>A system level configuration carbon.lucene.compression.mode can be added for best compression of
+lucene index files. The default value is speed, where the index writing speed will be more. If the
+value is compression, the index file size will be compressed.</p>
+<h2>
+<a id="querying-data" class="anchor" href="#querying-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Querying data</h2>
+<p>As a technique for query acceleration, Lucene indexes cannot be queried directly.
+Queries are to be made on main table. when a query with TEXT_MATCH('name:c10') or
+TEXT_MATCH_WITH_LIMIT('name:n10',10)[the second parameter represents the number of result to be
+returned, if user does not specify this value, all results will be returned without any limit] is
+fired, two jobs are fired.The first job writes the temporary files in folder created at table level
+which contains lucene's seach results and these files will be read in second job to give faster
+results. These temporary files will be cleared once the query finishes.</p>
+<p>User can verify whether a query can leverage Lucene datamap or not by executing <code>EXPLAIN</code>
+command, which will show the transformed logical plan, and thus user can check whether TEXT_MATCH()
+filter is applied on query or not.</p>
+<p><strong>Note:</strong></p>
+<ol>
+<li>
+<p>The filter columns in TEXT_MATCH or TEXT_MATCH_WITH_LIMIT must be always in lower case and
+filter condition like 'AND','OR' must be in upper case.</p>
+<p>Ex:</p>
+<pre><code>select * from datamap_test where TEXT_MATCH('name:*10 AND name:*n*')
+</code></pre>
+</li>
+<li>
+<p>Query supports only one TEXT_MATCH udf for filter condition and not multiple udfs.</p>
+<p>The following query is supported:</p>
+<pre><code>select * from datamap_test where TEXT_MATCH('name:*10 AND name:*n*')
+</code></pre>
+<p>The following query is not supported:</p>
+<pre><code>select * from datamap_test where TEXT_MATCH('name:*10) AND TEXT_MATCH(name:*n*')
+</code></pre>
+</li>
+</ol>
+<p>Below like queries can be converted to text_match queries as following:</p>
+<pre><code>select * from datamap_test where name='n10'
+
+select * from datamap_test where name like 'n1%'
+
+select * from datamap_test where name like '%10'
+
+select * from datamap_test where name like '%n%'
+
+select * from datamap_test where name like '%10' and name not like '%n%'
+</code></pre>
+<p>Lucene TEXT_MATCH Queries:</p>
+<pre><code>select * from datamap_test where TEXT_MATCH('name:n10')
+
+select * from datamap_test where TEXT_MATCH('name:n1*')
+
+select * from datamap_test where TEXT_MATCH('name:*10')
+
+select * from datamap_test where TEXT_MATCH('name:*n*')
+
+select * from datamap_test where TEXT_MATCH('name:*10 -name:*n*')
+</code></pre>
+<p><strong>Note:</strong> For lucene queries and syntax, refer to <a href="www.lucenetutorial.com/lucene-query-syntax.html">lucene-syntax</a></p>
+<h2>
+<a id="data-management-with-lucene-datamap" class="anchor" href="#data-management-with-lucene-datamap" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Management with lucene datamap</h2>
+<p>Once there is lucene datamap is created on the main table, following command on the main
+table
+is not supported:</p>
+<ol>
+<li>Data management command: <code>UPDATE/DELETE</code>.</li>
+<li>Schema management command: <code>ALTER TABLE DROP COLUMN</code>, <code>ALTER TABLE CHANGE DATATYPE</code>,
+<code>ALTER TABLE RENAME</code>.</li>
+</ol>
+<p><strong>Note</strong>: Adding a new column is supported, and for dropping columns and change datatype
+command, CarbonData will check whether it will impact the lucene datamap, if not, the operation
+is allowed, otherwise operation will be rejected by throwing exception.</p>
+<ol start="3">
+<li>Partition management command: <code>ALTER TABLE ADD/DROP PARTITION</code>.</li>
+</ol>
+<p>However, there is still way to support these operations on main table, in current CarbonData
+release, user can do as following:</p>
+<ol>
+<li>Remove the lucene datamap by <code>DROP DATAMAP</code> command.</li>
+<li>Carry out the data management operation on main table.</li>
+<li>Create the lucene datamap again by <code>CREATE DATAMAP</code> command.
+Basically, user can manually trigger the operation by re-building the datamap.</li>
+</ol>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/mainpage.html
----------------------------------------------------------------------
diff --git a/content/mainpage.html b/content/mainpage.html
index 7a5501b..d2746db 100644
--- a/content/mainpage.html
+++ b/content/mainpage.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>
@@ -188,17 +191,16 @@
                                     <div>
                                         <ul class="sub-nav">
                                             <li><a href="quick-start-guide.html">Quick Start</a></li>
-                                            <li><a href="file-structure-of-carbondata.html">CarbonData
-                                                File Structure</a></li>
-                                            <li><a href="supported-data-types-in-carbondata.html">Data
-                                                Types</a></li>
-                                            <li><a href="data-management-on-carbondata.html">Data
-                                                Management On CarbonData</a></li>
+                                            <li><a href="file-structure-of-carbondata.html">CarbonData File Structure</a></li>
+                                            <li><a href="supported-data-types-in-carbondata.html">Data Types</a></li>
+                                            <li><a href="data-management-on-carbondata.html">Data Management On CarbonData</a></li>
                                             <li><a href="installation-guide.html">Installation Guide</a></li>
                                             <li><a href="configuration-parameters.html">Configuring CarbonData</a></li>
                                             <li><a href="streaming-guide.html">Streaming Guide</a></li>
-                                            <li><a href="sdk-writer-guide.html">SDK Writer Guide</a></li>
+                                            <li><a href="sdk-guide.html">SDK Writer Guide</a></li>
                                             <li><a href="datamap-developer-guide.html">DataMap Developer Guide</a></li>
+                                            <li><a href="bloomfilter-datamap-guide.html">CarbonData Pre-aggregate DataMap</a></li>
+                                            <li><a href="lucene-datamap-guide.html">CarbonData Pre-aggregate DataMap</a></li>
                                             <li><a href="preaggregate-datamap-guide.html">CarbonData Pre-aggregate DataMap</a></li>
                                             <li><a href="timeseries-datamap-guide.html">CarbonData Timeseries DataMap</a></li>
                                             <li><a href="faq.html">FAQs</a></li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/pdf/CarbonData Documentation.pdf
----------------------------------------------------------------------
diff --git a/content/pdf/CarbonData Documentation.pdf b/content/pdf/CarbonData Documentation.pdf
index 544de04..a557cca 100644
Binary files a/content/pdf/CarbonData Documentation.pdf and b/content/pdf/CarbonData Documentation.pdf differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/preaggregate-datamap-guide.html
----------------------------------------------------------------------
diff --git a/content/preaggregate-datamap-guide.html b/content/preaggregate-datamap-guide.html
index 35a0f62..d195bfe 100644
--- a/content/preaggregate-datamap-guide.html
+++ b/content/preaggregate-datamap-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/content/quick-start-guide.html
----------------------------------------------------------------------
diff --git a/content/quick-start-guide.html b/content/quick-start-guide.html
index 002df87..20cfbf1 100644
--- a/content/quick-start-guide.html
+++ b/content/quick-start-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>


[2/5] carbondata-site git commit: doucment for latest release of carbondata 1.4.0

Posted by ch...@apache.org.
http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/lucene-datamap-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/lucene-datamap-guide.html b/src/main/webapp/lucene-datamap-guide.html
new file mode 100644
index 0000000..5f56429
--- /dev/null
+++ b/src/main/webapp/lucene-datamap-guide.html
@@ -0,0 +1,337 @@
+<!DOCTYPE html>
+<html lang="en">
+<head>
+    <meta charset="utf-8">
+    <meta http-equiv="X-UA-Compatible" content="IE=edge">
+    <meta name="viewport" content="width=device-width, initial-scale=1">
+    <link href='images/favicon.ico' rel='shortcut icon' type='image/x-icon'>
+    <!-- The above 3 meta tags *must* come first in the head; any other head content must come *after* these tags -->
+    <title>CarbonData</title>
+    <style>
+
+    </style>
+    <!-- Bootstrap -->
+
+    <link rel="stylesheet" href="css/bootstrap.min.css">
+    <link href="css/style.css" rel="stylesheet">
+    <!-- HTML5 shim and Respond.js for IE8 support of HTML5 elements and media queries -->
+    <!-- WARNING: Respond.js doesn't work if you view the page via file:// -->
+    <!--[if lt IE 9]>
+    <script src="https://oss.maxcdn.com/html5shiv/3.7.3/html5shiv.min.js"></script>
+    <script src="https://oss.maxcdn.scom/respond/1.4.2/respond.min.js"></script>
+    <![endif]-->
+    <script src="js/jquery.min.js"></script>
+    <script src="js/bootstrap.min.js"></script>
+
+
+</head>
+<body>
+<header>
+    <nav class="navbar navbar-default navbar-custom cd-navbar-wrapper">
+        <div class="container">
+            <div class="navbar-header">
+                <button aria-controls="navbar" aria-expanded="false" data-target="#navbar" data-toggle="collapse"
+                        class="navbar-toggle collapsed" type="button">
+                    <span class="sr-only">Toggle navigation</span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                    <span class="icon-bar"></span>
+                </button>
+                <a href="index.html" class="logo">
+                    <img src="images/CarbonDataLogo.png" alt="CarbonData logo" title="CarbocnData logo"/>
+                </a>
+            </div>
+            <div class="navbar-collapse collapse cd_navcontnt" id="navbar">
+                <ul class="nav navbar-nav navbar-right navlist-custom">
+                    <li><a href="index.html" class="hidden-xs"><i class="fa fa-home" aria-hidden="true"></i> </a>
+                    </li>
+                    <li><a href="index.html" class="hidden-lg hidden-md hidden-sm">Home</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle " data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false"> Download <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
+                                   target="_blank">Apache CarbonData 1.3.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.0/"
+                                   target="_blank">Apache CarbonData 1.3.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.2.0/"
+                                   target="_blank">Apache CarbonData 1.2.0</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.1/"
+                                   target="_blank">Apache CarbonData 1.1.1</a></li>
+                            <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.1.0/"
+                                   target="_blank">Apache CarbonData 1.1.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/1.0.0-incubating/"
+                                   target="_blank">Apache CarbonData 1.0.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.2.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.2.0</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.1-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.1</a></li>
+                            <li>
+                                <a href="http://archive.apache.org/dist/incubator/carbondata/0.1.0-incubating/"
+                                   target="_blank">Apache CarbonData 0.1.0</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/Releases"
+                                   target="_blank">Release Archive</a></li>
+                        </ul>
+                    </li>
+                    <li><a href="mainpage.html" class="active">Documentation</a></li>
+                    <li class="dropdown">
+                        <a href="#" class="dropdown-toggle" data-toggle="dropdown" role="button" aria-haspopup="true"
+                           aria-expanded="false">Community <span class="caret"></span></a>
+                        <ul class="dropdown-menu">
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/How-to-contribute-to-Apache-CarbonData.md"
+                                   target="_blank">Contributing to CarbonData</a></li>
+                            <li>
+                                <a href="https://github.com/apache/carbondata/blob/master/docs/release-guide.md"
+                                   target="_blank">Release Guide</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/display/CARBONDATA/PMC+and+Committers+member+list"
+                                   target="_blank">Project PMC and Committers</a></li>
+                            <li>
+                                <a href="https://cwiki.apache.org/confluence/pages/viewpage.action?pageId=66850609"
+                                   target="_blank">CarbonData Meetups</a></li>
+                            <li><a href="security.html">Apache CarbonData Security</a></li>
+                            <li><a href="https://issues.apache.org/jira/browse/CARBONDATA" target="_blank">Apache
+                                Jira</a></li>
+                            <li><a href="videogallery.html">CarbonData Videos </a></li>
+                        </ul>
+                    </li>
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="apache_link hidden-xs dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li class="dropdown">
+                        <a href="http://www.apache.org/" class="hidden-lg hidden-md hidden-sm dropdown-toggle"
+                           data-toggle="dropdown" role="button" aria-haspopup="true" aria-expanded="false">Apache</a>
+                        <ul class="dropdown-menu">
+                            <li><a href="http://www.apache.org/" target="_blank">Apache Homepage</a></li>
+                            <li><a href="http://www.apache.org/licenses/" target="_blank">License</a></li>
+                            <li><a href="http://www.apache.org/foundation/sponsorship.html"
+                                   target="_blank">Sponsorship</a></li>
+                            <li><a href="http://www.apache.org/foundation/thanks.html" target="_blank">Thanks</a></li>
+                        </ul>
+                    </li>
+
+                    <li>
+                        <a href="#" id="search-icon"><i class="fa fa-search" aria-hidden="true"></i></a>
+
+                    </li>
+
+                </ul>
+            </div><!--/.nav-collapse -->
+            <div id="search-box">
+                <form method="get" action="http://www.google.com/search" target="_blank">
+                    <div class="search-block">
+                        <table border="0" cellpadding="0" width="100%">
+                            <tr>
+                                <td style="width:80%">
+                                    <input type="text" name="q" size=" 5" maxlength="255" value=""
+                                           class="search-input"  placeholder="Search...."    required/>
+                                </td>
+                                <td style="width:20%">
+                                    <input type="submit" value="Search"/></td>
+                            </tr>
+                            <tr>
+                                <td align="left" style="font-size:75%" colspan="2">
+                                    <input type="checkbox" name="sitesearch" value="carbondata.apache.org" checked/>
+                                    <span style=" position: relative; top: -3px;"> Only search for CarbonData</span>
+                                </td>
+                            </tr>
+                        </table>
+                    </div>
+                </form>
+            </div>
+        </div>
+    </nav>
+</header> <!-- end Header part -->
+
+<div class="fixed-padding"></div> <!--  top padding with fixde header  -->
+
+<section><!-- Dashboard nav -->
+    <div class="container-fluid q">
+        <div class="col-sm-12  col-md-12 maindashboard">
+            <div class="row">
+                <section>
+                    <div style="padding:10px 15px;">
+                        <div id="viewpage" name="viewpage">
+                            <div class="row">
+                                <div class="col-sm-12  col-md-12">
+                                    <div><h1>
+<a id="carbondata-lucene-datamap-alpha-feature-in-140" class="anchor" href="#carbondata-lucene-datamap-alpha-feature-in-140" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>CarbonData Lucene DataMap (Alpha feature in 1.4.0)</h1>
+<ul>
+<li><a href="#datamap-management">DataMap Management</a></li>
+<li><a href="#lucene-datamap-introduction">Lucene Datamap</a></li>
+<li><a href="#loading-data">Loading Data</a></li>
+<li><a href="#querying-data">Querying Data</a></li>
+<li><a href="#data-management-with-lucene-datamap">Data Management</a></li>
+</ul>
+<h4>
+<a id="datamap-management" class="anchor" href="#datamap-management" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>DataMap Management</h4>
+<p>Lucene DataMap can be created using following DDL</p>
+<pre><code>CREATE DATAMAP [IF NOT EXISTS] datamap_name
+ON TABLE main_table
+USING 'lucene'
+DMPROPERTIES ('index_columns'='city, name', ...)
+</code></pre>
+<p>DataMap can be dropped using following DDL:</p>
+<pre><code>DROP DATAMAP [IF EXISTS] datamap_name
+ON TABLE main_table
+</code></pre>
+<p>To show all DataMaps created, use:</p>
+<pre><code>SHOW DATAMAP 
+ON TABLE main_table
+</code></pre>
+<p>It will show all DataMaps created on main table.</p>
+<h2>
+<a id="lucene-datamap-introduction" class="anchor" href="#lucene-datamap-introduction" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Lucene DataMap Introduction</h2>
+<p>Lucene is a high performance, full featured text search engine. Lucene is integrated to carbon as
+an index datamap and managed along with main tables by CarbonData.User can create lucene datamap
+to improve query performance on string columns which has content of more length. So, user can
+search tokenized word or pattern of it using lucene query on text content.</p>
+<p>For instance, main table called <strong>datamap_test</strong> which is defined as:</p>
+<pre><code>CREATE TABLE datamap_test (
+  name string,
+  age int,
+  city string,
+  country string)
+STORED BY 'carbondata'
+</code></pre>
+<p>User can create Lucene datamap using the Create DataMap DDL:</p>
+<pre><code>CREATE DATAMAP dm
+ON TABLE datamap_test
+USING 'lucene'
+DMPROPERTIES ('INDEX_COLUMNS' = 'name, country',)
+</code></pre>
+<p><strong>DMProperties</strong></p>
+<ol>
+<li>INDEX_COLUMNS: The list of string columns on which lucene creates indexes.</li>
+<li>FLUSH_CACHE: size of the cache to maintain in Lucene writer, if specified then it tries to
+aggregate the unique data till the cache limit and flush to Lucene. It is best suitable for low
+cardinality dimensions.</li>
+<li>SPLIT_BLOCKLET: when made as true then store the data in blocklet wise in lucene , it means new
+folder will be created for each blocklet, thus, it eliminates storing blockletid in lucene and
+also it makes lucene small chunks of data.</li>
+</ol>
+<h2>
+<a id="loading-data" class="anchor" href="#loading-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Loading data</h2>
+<p>When loading data to main table, lucene index files will be generated for all the
+index_columns(String Columns) given in DMProperties which contains information about the data
+location of index_columns. These index files will be written inside a folder named with datamap name
+inside each segment folders.</p>
+<p>A system level configuration carbon.lucene.compression.mode can be added for best compression of
+lucene index files. The default value is speed, where the index writing speed will be more. If the
+value is compression, the index file size will be compressed.</p>
+<h2>
+<a id="querying-data" class="anchor" href="#querying-data" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Querying data</h2>
+<p>As a technique for query acceleration, Lucene indexes cannot be queried directly.
+Queries are to be made on main table. when a query with TEXT_MATCH('name:c10') or
+TEXT_MATCH_WITH_LIMIT('name:n10',10)[the second parameter represents the number of result to be
+returned, if user does not specify this value, all results will be returned without any limit] is
+fired, two jobs are fired.The first job writes the temporary files in folder created at table level
+which contains lucene's seach results and these files will be read in second job to give faster
+results. These temporary files will be cleared once the query finishes.</p>
+<p>User can verify whether a query can leverage Lucene datamap or not by executing <code>EXPLAIN</code>
+command, which will show the transformed logical plan, and thus user can check whether TEXT_MATCH()
+filter is applied on query or not.</p>
+<p><strong>Note:</strong></p>
+<ol>
+<li>
+<p>The filter columns in TEXT_MATCH or TEXT_MATCH_WITH_LIMIT must be always in lower case and
+filter condition like 'AND','OR' must be in upper case.</p>
+<p>Ex:</p>
+<pre><code>select * from datamap_test where TEXT_MATCH('name:*10 AND name:*n*')
+</code></pre>
+</li>
+<li>
+<p>Query supports only one TEXT_MATCH udf for filter condition and not multiple udfs.</p>
+<p>The following query is supported:</p>
+<pre><code>select * from datamap_test where TEXT_MATCH('name:*10 AND name:*n*')
+</code></pre>
+<p>The following query is not supported:</p>
+<pre><code>select * from datamap_test where TEXT_MATCH('name:*10) AND TEXT_MATCH(name:*n*')
+</code></pre>
+</li>
+</ol>
+<p>Below like queries can be converted to text_match queries as following:</p>
+<pre><code>select * from datamap_test where name='n10'
+
+select * from datamap_test where name like 'n1%'
+
+select * from datamap_test where name like '%10'
+
+select * from datamap_test where name like '%n%'
+
+select * from datamap_test where name like '%10' and name not like '%n%'
+</code></pre>
+<p>Lucene TEXT_MATCH Queries:</p>
+<pre><code>select * from datamap_test where TEXT_MATCH('name:n10')
+
+select * from datamap_test where TEXT_MATCH('name:n1*')
+
+select * from datamap_test where TEXT_MATCH('name:*10')
+
+select * from datamap_test where TEXT_MATCH('name:*n*')
+
+select * from datamap_test where TEXT_MATCH('name:*10 -name:*n*')
+</code></pre>
+<p><strong>Note:</strong> For lucene queries and syntax, refer to <a href="www.lucenetutorial.com/lucene-query-syntax.html">lucene-syntax</a></p>
+<h2>
+<a id="data-management-with-lucene-datamap" class="anchor" href="#data-management-with-lucene-datamap" aria-hidden="true"><span aria-hidden="true" class="octicon octicon-link"></span></a>Data Management with lucene datamap</h2>
+<p>Once there is lucene datamap is created on the main table, following command on the main
+table
+is not supported:</p>
+<ol>
+<li>Data management command: <code>UPDATE/DELETE</code>.</li>
+<li>Schema management command: <code>ALTER TABLE DROP COLUMN</code>, <code>ALTER TABLE CHANGE DATATYPE</code>,
+<code>ALTER TABLE RENAME</code>.</li>
+</ol>
+<p><strong>Note</strong>: Adding a new column is supported, and for dropping columns and change datatype
+command, CarbonData will check whether it will impact the lucene datamap, if not, the operation
+is allowed, otherwise operation will be rejected by throwing exception.</p>
+<ol start="3">
+<li>Partition management command: <code>ALTER TABLE ADD/DROP PARTITION</code>.</li>
+</ol>
+<p>However, there is still way to support these operations on main table, in current CarbonData
+release, user can do as following:</p>
+<ol>
+<li>Remove the lucene datamap by <code>DROP DATAMAP</code> command.</li>
+<li>Carry out the data management operation on main table.</li>
+<li>Create the lucene datamap again by <code>CREATE DATAMAP</code> command.
+Basically, user can manually trigger the operation by re-building the datamap.</li>
+</ol>
+</div>
+</div>
+</div>
+</div>
+<div class="doc-footer">
+    <a href="#top" class="scroll-top">Top</a>
+</div>
+</div>
+</section>
+</div>
+</div>
+</div>
+</section><!-- End systemblock part -->
+<script src="js/custom.js"></script>
+</body>
+</html>
\ No newline at end of file

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/mainpage.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/mainpage.html b/src/main/webapp/mainpage.html
index 7a5501b..39abe9b 100644
--- a/src/main/webapp/mainpage.html
+++ b/src/main/webapp/mainpage.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>
@@ -188,17 +191,16 @@
                                     <div>
                                         <ul class="sub-nav">
                                             <li><a href="quick-start-guide.html">Quick Start</a></li>
-                                            <li><a href="file-structure-of-carbondata.html">CarbonData
-                                                File Structure</a></li>
-                                            <li><a href="supported-data-types-in-carbondata.html">Data
-                                                Types</a></li>
-                                            <li><a href="data-management-on-carbondata.html">Data
-                                                Management On CarbonData</a></li>
+                                            <li><a href="file-structure-of-carbondata.html">CarbonData File Structure</a></li>
+                                            <li><a href="supported-data-types-in-carbondata.html">Data Types</a></li>
+                                            <li><a href="data-management-on-carbondata.html">Data Management On CarbonData</a></li>
                                             <li><a href="installation-guide.html">Installation Guide</a></li>
                                             <li><a href="configuration-parameters.html">Configuring CarbonData</a></li>
                                             <li><a href="streaming-guide.html">Streaming Guide</a></li>
-                                            <li><a href="sdk-writer-guide.html">SDK Writer Guide</a></li>
+                                            <li><a href="sdk-guide.html">SDK Guide</a></li>
                                             <li><a href="datamap-developer-guide.html">DataMap Developer Guide</a></li>
+                                            <li><a href="bloomfilter-datamap-guide.html">CarbonData BloomFilter DataMap (Alpha feature in 1.4.0)</a></li>
+                                            <li><a href="lucene-datamap-guide.html">CarbonData Lucene DataMap (Alpha feature in 1.4.0)</a></li>
                                             <li><a href="preaggregate-datamap-guide.html">CarbonData Pre-aggregate DataMap</a></li>
                                             <li><a href="timeseries-datamap-guide.html">CarbonData Timeseries DataMap</a></li>
                                             <li><a href="faq.html">FAQs</a></li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/pdf/CarbonData Documentation.pdf
----------------------------------------------------------------------
diff --git a/src/main/webapp/pdf/CarbonData Documentation.pdf b/src/main/webapp/pdf/CarbonData Documentation.pdf
index 544de04..a557cca 100644
Binary files a/src/main/webapp/pdf/CarbonData Documentation.pdf and b/src/main/webapp/pdf/CarbonData Documentation.pdf differ

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/preaggregate-datamap-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/preaggregate-datamap-guide.html b/src/main/webapp/preaggregate-datamap-guide.html
index 35a0f62..d195bfe 100644
--- a/src/main/webapp/preaggregate-datamap-guide.html
+++ b/src/main/webapp/preaggregate-datamap-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/quick-start-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/quick-start-guide.html b/src/main/webapp/quick-start-guide.html
index 002df87..20cfbf1 100644
--- a/src/main/webapp/quick-start-guide.html
+++ b/src/main/webapp/quick-start-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

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

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/sdk-writer-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/sdk-writer-guide.html b/src/main/webapp/sdk-writer-guide.html
index a73c22c..36bb9ad 100644
--- a/src/main/webapp/sdk-writer-guide.html
+++ b/src/main/webapp/sdk-writer-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/security.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/security.html b/src/main/webapp/security.html
index 826ed4a..2b0ab61 100644
--- a/src/main/webapp/security.html
+++ b/src/main/webapp/security.html
@@ -45,6 +45,9 @@
                            aria-expanded="false">Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/streaming-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/streaming-guide.html b/src/main/webapp/streaming-guide.html
index e9788a9..981a095 100644
--- a/src/main/webapp/streaming-guide.html
+++ b/src/main/webapp/streaming-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>
@@ -194,11 +197,11 @@
  <span class="pl-k">import</span> <span class="pl-en">org</span>.<span class="pl-en">apache</span>.<span class="pl-en">spark</span>.<span class="pl-en">sql</span>.{<span class="pl-en">CarbonEnv</span>, <span class="pl-en">SparkSession</span>}
  <span class="pl-k">import</span> <span class="pl-en">org</span>.<span class="pl-en">apache</span>.<span class="pl-en">spark</span>.<span class="pl-en">sql</span>.<span class="pl-en">CarbonSession</span>.<span class="pl-en">_</span>
  <span class="pl-k">import</span> <span class="pl-en">org</span>.<span class="pl-en">apache</span>.<span class="pl-en">spark</span>.<span class="pl-en">sql</span>.<span class="pl-en">streaming</span>.{<span class="pl-en">ProcessingTime</span>, <span class="pl-en">StreamingQuery</span>}
- <span class="pl-k">import</span> <span class="pl-en">org</span>.<span class="pl-en">apache</span>.<span class="pl-en">carbondata</span>.<span class="pl-en">core</span>.<span class="pl-en">util</span>.<span class="pl-en">path</span>.<span class="pl-en">CarbonStorePath</span>
- 
+ <span class="pl-k">import</span> <span class="pl-en">org</span>.<span class="pl-en">apache</span>.<span class="pl-en">carbondata</span>.<span class="pl-en">core</span>.<span class="pl-en">util</span>.<span class="pl-en">path</span>.<span class="pl-en">CarbonTablePath</span>
+
  <span class="pl-k">val</span> <span class="pl-smi">warehouse</span> <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-en">File</span>(<span class="pl-s"><span class="pl-pds">"</span>./warehouse<span class="pl-pds">"</span></span>).getCanonicalPath
  <span class="pl-k">val</span> <span class="pl-smi">metastore</span> <span class="pl-k">=</span> <span class="pl-k">new</span> <span class="pl-en">File</span>(<span class="pl-s"><span class="pl-pds">"</span>./metastore<span class="pl-pds">"</span></span>).getCanonicalPath
- 
+
  <span class="pl-k">val</span> <span class="pl-smi">spark</span> <span class="pl-k">=</span> <span class="pl-en">SparkSession</span>
    .builder()
    .master(<span class="pl-s"><span class="pl-pds">"</span>local<span class="pl-pds">"</span></span>)
@@ -221,8 +224,8 @@
 <span class="pl-s">      | TBLPROPERTIES('streaming'='true')<span class="pl-pds">"""</span></span>.stripMargin)
 
  <span class="pl-k">val</span> <span class="pl-smi">carbonTable</span> <span class="pl-k">=</span> <span class="pl-en">CarbonEnv</span>.getCarbonTable(<span class="pl-en">Some</span>(<span class="pl-s"><span class="pl-pds">"</span>default<span class="pl-pds">"</span></span>), <span class="pl-s"><span class="pl-pds">"</span>carbon_table<span class="pl-pds">"</span></span>)(spark)
- <span class="pl-k">val</span> <span class="pl-smi">tablePath</span> <span class="pl-k">=</span> <span class="pl-en">CarbonStorePath</span>.getCarbonTablePath(carbonTable.getAbsoluteTableIdentifier)
- 
+ <span class="pl-k">val</span> <span class="pl-smi">tablePath</span> <span class="pl-k">=</span> carbonTable.getTablePath
+
  <span class="pl-c"><span class="pl-c">//</span> batch load</span>
  <span class="pl-k">var</span> <span class="pl-smi">qry</span><span class="pl-k">:</span> <span class="pl-en">StreamingQuery</span> <span class="pl-k">=</span> <span class="pl-c1">null</span>
  <span class="pl-k">val</span> <span class="pl-smi">readSocketDF</span> <span class="pl-k">=</span> spark.readStream
@@ -235,7 +238,7 @@
  qry <span class="pl-k">=</span> readSocketDF.writeStream
    .format(<span class="pl-s"><span class="pl-pds">"</span>carbondata<span class="pl-pds">"</span></span>)
    .trigger(<span class="pl-en">ProcessingTime</span>(<span class="pl-s"><span class="pl-pds">"</span>5 seconds<span class="pl-pds">"</span></span>))
-   .option(<span class="pl-s"><span class="pl-pds">"</span>checkpointLocation<span class="pl-pds">"</span></span>, tablePath.getStreamingCheckpointDir)
+   .option(<span class="pl-s"><span class="pl-pds">"</span>checkpointLocation<span class="pl-pds">"</span></span>, <span class="pl-en">CarbonTablePath</span>.getStreamingCheckpointDir(tablePath))
    .option(<span class="pl-s"><span class="pl-pds">"</span>dbName<span class="pl-pds">"</span></span>, <span class="pl-s"><span class="pl-pds">"</span>default<span class="pl-pds">"</span></span>)
    .option(<span class="pl-s"><span class="pl-pds">"</span>tableName<span class="pl-pds">"</span></span>, <span class="pl-s"><span class="pl-pds">"</span>carbon_table<span class="pl-pds">"</span></span>)
    .start()

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/supported-data-types-in-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/supported-data-types-in-carbondata.html b/src/main/webapp/supported-data-types-in-carbondata.html
index 43d1071..0f06a42 100644
--- a/src/main/webapp/supported-data-types-in-carbondata.html
+++ b/src/main/webapp/supported-data-types-in-carbondata.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>
@@ -212,6 +215,7 @@
 <li>structs: STRUCT<code>&lt;col_name : data_type COMMENT col_comment, ...&gt;</code>
 </li>
 </ul>
+<p><strong>NOTE</strong>: Only 2 level complex type schema is supported for now.</p>
 </li>
 <li>
 <p>Other Types</p>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/timeseries-datamap-guide.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/timeseries-datamap-guide.html b/src/main/webapp/timeseries-datamap-guide.html
index 300e879..b40150c 100644
--- a/src/main/webapp/timeseries-datamap-guide.html
+++ b/src/main/webapp/timeseries-datamap-guide.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/troubleshooting.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/troubleshooting.html b/src/main/webapp/troubleshooting.html
index 37a07b4..d5cf1b2 100644
--- a/src/main/webapp/troubleshooting.html
+++ b/src/main/webapp/troubleshooting.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/useful-tips-on-carbondata.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/useful-tips-on-carbondata.html b/src/main/webapp/useful-tips-on-carbondata.html
index 8b653b4..e8b0ab0 100644
--- a/src/main/webapp/useful-tips-on-carbondata.html
+++ b/src/main/webapp/useful-tips-on-carbondata.html
@@ -51,6 +51,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/main/webapp/videogallery.html
----------------------------------------------------------------------
diff --git a/src/main/webapp/videogallery.html b/src/main/webapp/videogallery.html
index 2dd8c67..3c9e5eb 100644
--- a/src/main/webapp/videogallery.html
+++ b/src/main/webapp/videogallery.html
@@ -49,6 +49,9 @@
                            aria-expanded="false"> Download <span class="caret"></span></a>
                         <ul class="dropdown-menu">
                             <li>
+                                <a href="https://dist.apache.org/repos/dist/release/carbondata/1.4.0/"
+                                   target="_blank">Apache CarbonData 1.4.0</a></li>
+                            <li>
                                 <a href="https://dist.apache.org/repos/dist/release/carbondata/1.3.1/"
                                    target="_blank">Apache CarbonData 1.3.1</a></li>
                             <li>

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/site/markdown/bloomfilter-datamap-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/bloomfilter-datamap-guide.md b/src/site/markdown/bloomfilter-datamap-guide.md
new file mode 100644
index 0000000..fc46f54
--- /dev/null
+++ b/src/site/markdown/bloomfilter-datamap-guide.md
@@ -0,0 +1,98 @@
+# CarbonData BloomFilter DataMap (Alpha feature in 1.4.0)
+
+* [DataMap Management](#datamap-management)
+* [BloomFilter Datamap Introduction](#bloomfilter-datamap-introduction)
+* [Loading Data](#loading-data)
+* [Querying Data](#querying-data)
+* [Data Management](#data-management-with-bloomfilter-datamap)
+
+#### DataMap Management
+Creating BloomFilter DataMap
+  ```
+  CREATE DATAMAP [IF NOT EXISTS] datamap_name
+  ON TABLE main_table
+  USING 'bloomfilter'
+  DMPROPERTIES ('index_columns'='city, name', 'BLOOM_SIZE'='640000', 'BLOOM_FPP'='0.00001')
+  ```
+
+Dropping specified datamap
+  ```
+  DROP DATAMAP [IF EXISTS] datamap_name
+  ON TABLE main_table
+  ```
+
+Showing all DataMaps on this table
+  ```
+  SHOW DATAMAP
+  ON TABLE main_table
+  ```
+It will show all DataMaps created on main table.
+
+
+## BloomFilter DataMap Introduction
+A Bloom filter is a space-efficient probabilistic data structure that is used to test whether an element is a member of a set.
+Carbondata introduce BloomFilter as an index datamap to enhance the performance of querying with precise value.
+It is well suitable for queries that do precise match on high cardinality columns(such as Name/ID).
+Internally, CarbonData maintains a BloomFilter per blocklet for each index column to indicate that whether a value of the column is in this blocklet.
+Just like the other datamaps, BloomFilter datamap is managed ablong with main tables by CarbonData.
+User can create BloomFilter datamap on specified columns with specified BloomFilter configurations such as size and probability.
+
+For instance, main table called **datamap_test** which is defined as:
+
+  ```
+  CREATE TABLE datamap_test (
+    id string,
+    name string,
+    age int,
+    city string,
+    country string)
+  STORED BY 'carbondata'
+  TBLPROPERTIES('SORT_COLUMNS'='id')
+  ```
+
+In the above example, `id` and `name` are high cardinality columns
+and we always query on `id` and `name` with precise value.
+since `id` is in the sort_columns and it is orderd,
+query on it will be fast because CarbonData can skip all the irrelative blocklets.
+But queries on `name` may be bad since the blocklet minmax may not help,
+because in each blocklet the range of the value of `name` may be the same -- all from A*~z*.
+In this case, user can create a BloomFilter datamap on column `name`.
+Moreover, user can also create a BloomFilter datamap on the sort_columns.
+This is useful if user has too many segments and the range of the value of sort_columns are almost the same.
+
+User can create BloomFilter datamap using the Create DataMap DDL:
+
+  ```
+  CREATE DATAMAP dm
+  ON TABLE datamap_test
+  USING 'bloomfilter'
+  DMPROPERTIES ('INDEX_COLUMNS' = 'name,id', 'BLOOM_SIZE'='640000', 'BLOOM_FPP'='0.00001', 'BLOOM_COMPRESS'='true')
+  ```
+
+**Properties for BloomFilter DataMap**
+
+| Property | Is Required | Default Value | Description |
+|-------------|----------|--------|---------|
+| INDEX_COLUMNS | YES |  | Carbondata will generate BloomFilter index on these columns. Queries on there columns are usually like 'COL = VAL'. |
+| BLOOM_SIZE | NO | 32000 | This value is internally used by BloomFilter as the number of expected insertions, it will affects the size of BloomFilter index. Since each blocklet has a BloomFilter here, so the value is the approximate records in a blocklet. In another word, the value 32000 * #noOfPagesInBlocklet. The value should be an integer. |
+| BLOOM_FPP | NO | 0.01 | This value is internally used by BloomFilter as the False-Positive Probability, it will affects the size of bloomfilter index as well as the number of hash functions for the BloomFilter. The value should be in range (0, 1). |
+| BLOOM_COMPRESS | NO | true | Whether to compress the BloomFilter index files. |
+
+
+## Loading Data
+When loading data to main table, BloomFilter files will be generated for all the
+index_columns given in DMProperties which contains the blockletId and a BloomFilter for each index column.
+These index files will be written inside a folder named with datamap name
+inside each segment folders.
+
+
+## Querying Data
+A system level configuration `carbon.query.datamap.bloom.cache.size` can used to enhance query performance with BloomFilter datamap by providing a cache for the bloomfilter index files.
+The default value is `512` and its unit is `MB`. Internally the cache will be expired after it's idle for 2 hours.
+
+User can verify whether a query can leverage BloomFilter datamap by executing `EXPLAIN` command,
+which will show the transformed logical plan, and thus user can check whether the BloomFilter datamap can skip blocklets during the scan.
+If the datamap does not prune blocklets well, you can try to increase the value of property `BLOOM_SIZE` and decrease the value of property `BLOOM_FPP`.
+
+## Data Management With BloomFilter DataMap
+Data management with BloomFilter datamap has no difference with that on Lucene datamap. You can refer to the corresponding section in `CarbonData BloomFilter DataMap`.

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/2941aaca/src/site/markdown/configuration-parameters.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/configuration-parameters.md b/src/site/markdown/configuration-parameters.md
index e39d61b..11cc6ea 100644
--- a/src/site/markdown/configuration-parameters.md
+++ b/src/site/markdown/configuration-parameters.md
@@ -133,7 +133,8 @@ This section provides the details of all the configurations required for CarbonD
 | carbon.enableMinMax | true | Min max is feature added to enhance query performance. To disable this feature, set it false. |
 | carbon.dynamicallocation.schedulertimeout | 5 | Specifies the maximum time (unit in seconds) the scheduler can wait for executor to be active. Minimum value is 5 sec and maximum value is 15 sec. |
 | carbon.scheduler.minregisteredresourcesratio | 0.8 | Specifies the minimum resource (executor) ratio needed for starting the block distribution. The default value is 0.8, which indicates 80% of the requested resource is allocated for starting block distribution.  The minimum value is 0.1 min and the maximum value is 1.0. | 
-  
+| carbon.search.enabled | false | If set to true, it will use CarbonReader to do distributed scan directly instead of using compute framework like spark, thus avoiding limitation of compute framework like SQL optimizer and task scheduling overhead. |
+
 * **Global Dictionary Configurations**
   
 | Parameter | Default Value | Description |