You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by gv...@apache.org on 2018/06/08 11:40:51 UTC

[26/50] [abbrv] carbondata git commit: [CARBONDATA-2519] Add document for CarbonReader

[CARBONDATA-2519] Add document for CarbonReader

Add document for CarbonReader

This closes #2337


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

Branch: refs/heads/spark-2.3
Commit: ddf3e85952d066cb82f4963c5f7bce6a8196f095
Parents: d8bafa3
Author: xubo245 <xu...@huawei.com>
Authored: Wed May 23 21:45:49 2018 +0800
Committer: kunal642 <ku...@gmail.com>
Committed: Mon May 28 16:04:01 2018 +0530

----------------------------------------------------------------------
 README.md                                       |   2 +-
 docs/sdk-guide.md                               | 591 +++++++++++++++++++
 docs/sdk-writer-guide.md                        | 400 -------------
 .../carbondata/sdk/file/CarbonReader.java       |   2 +-
 .../sdk/file/CarbonReaderBuilder.java           |  56 +-
 .../sdk/file/CarbonWriterBuilder.java           |   8 +-
 6 files changed, 642 insertions(+), 417 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/ddf3e859/README.md
----------------------------------------------------------------------
diff --git a/README.md b/README.md
index be3186c..d8f7226 100644
--- a/README.md
+++ b/README.md
@@ -52,7 +52,7 @@ CarbonData is built using Apache Maven, to [build CarbonData](https://github.com
 * [Cluster Installation and Deployment](https://github.com/apache/carbondata/blob/master/docs/installation-guide.md)
 * [Configuring Carbondata](https://github.com/apache/carbondata/blob/master/docs/configuration-parameters.md)
 * [Streaming Ingestion](https://github.com/apache/carbondata/blob/master/docs/streaming-guide.md)
-* [SDK Writer Guide](https://github.com/apache/carbondata/blob/master/docs/sdk-writer-guide.md)
+* [SDK Guide](https://github.com/apache/carbondata/blob/master/docs/sdk-guide.md)
 * [CarbonData Pre-aggregate DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/preaggregate-datamap-guide.md)
 * [CarbonData Timeseries DataMap](https://github.com/apache/carbondata/blob/master/docs/datamap/timeseries-datamap-guide.md)
 * [FAQ](https://github.com/apache/carbondata/blob/master/docs/faq.md)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ddf3e859/docs/sdk-guide.md
----------------------------------------------------------------------
diff --git a/docs/sdk-guide.md b/docs/sdk-guide.md
new file mode 100644
index 0000000..4d258f0
--- /dev/null
+++ b/docs/sdk-guide.md
@@ -0,0 +1,591 @@
+# 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);
+```
+
+```
+  /**
+   * Read carbondata file and return the schema
+   */
+  public static List<ColumnSchema> readSchemaInDataFile(String dataFilePath);
+```
+
+```
+ /**
+  * Read schema file and return table info object
+  */
+  public static TableInfo readSchemaFile(String schemaFilePath);
+```
+
+```
+  /**
+   * 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);
+```
+
+```
+  /**
+   * 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 true
+   *
+   * @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();
+```
+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/blob/ddf3e859/docs/sdk-writer-guide.md
----------------------------------------------------------------------
diff --git a/docs/sdk-writer-guide.md b/docs/sdk-writer-guide.md
deleted file mode 100644
index 3d9a3de..0000000
--- a/docs/sdk-writer-guide.md
+++ /dev/null
@@ -1,400 +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.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);
-```

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ddf3e859/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
index 6517e89..d85bf4b 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReader.java
@@ -102,7 +102,7 @@ public class CarbonReader<T> {
   }
 
   /**
-   * Read schmea file and return table info object
+   * Read schema file and return table info object
    */
   public static TableInfo readSchemaFile(String schemaFilePath) throws IOException {
     org.apache.carbondata.format.TableInfo tableInfo = CarbonUtil.readSchemaFile(schemaFilePath);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ddf3e859/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
index 946ea0f..c78cda0 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonReaderBuilder.java
@@ -50,26 +50,53 @@ public class CarbonReaderBuilder {
   private String tableName;
   private boolean isTransactionalTable = true;
 
+  /**
+   * Construct a CarbonReaderBuilder with table path and table name
+   *
+   * @param tablePath table path
+   * @param tableName table name
+   */
   CarbonReaderBuilder(String tablePath, String tableName) {
     this.tablePath = tablePath;
     this.tableName = tableName;
   }
 
+  /**
+   * Configure the projection column names of carbon reader
+   *
+   * @param projectionColumnNames projection column names
+   * @return CarbonReaderBuilder object
+   */
   public CarbonReaderBuilder projection(String[] projectionColumnNames) {
     Objects.requireNonNull(projectionColumnNames);
     this.projectionColumns = projectionColumnNames;
     return this;
   }
 
+  /**
+   * 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 true
+   *
+   * @param isTransactionalTable whether is transactional table or not
+   * @return CarbonReaderBuilder object
+   */
   public CarbonReaderBuilder isTransactionalTable(boolean isTransactionalTable) {
     Objects.requireNonNull(isTransactionalTable);
     this.isTransactionalTable = isTransactionalTable;
     return this;
   }
 
-  public CarbonReaderBuilder filter(Expression fileterExpression) {
-    Objects.requireNonNull(fileterExpression);
-    this.filterExpression = fileterExpression;
+  /**
+   * Configure the filter expression for carbon reader
+   *
+   * @param filterExpression filter expression
+   * @return CarbonReaderBuilder object
+   */
+  public CarbonReaderBuilder filter(Expression filterExpression) {
+    Objects.requireNonNull(filterExpression);
+    this.filterExpression = filterExpression;
     return this;
   }
 
@@ -78,7 +105,7 @@ public class CarbonReaderBuilder {
    *
    * @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
+   * @return CarbonWriterBuilder object
    */
   public CarbonReaderBuilder setAccessKey(String key, String value) {
     FileFactory.getConfiguration().set(key, value);
@@ -89,7 +116,7 @@ public class CarbonReaderBuilder {
    * Set the access key for S3.
    *
    * @param value the value of access key
-   * @return CarbonWriterBuilder
+   * @return CarbonWriterBuilder object
    */
   public CarbonReaderBuilder setAccessKey(String value) {
     return setAccessKey(Constants.ACCESS_KEY, value);
@@ -100,7 +127,7 @@ public class CarbonReaderBuilder {
    *
    * @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
+   * @return CarbonWriterBuilder object
    */
   public CarbonReaderBuilder setSecretKey(String key, String value) {
     FileFactory.getConfiguration().set(key, value);
@@ -111,7 +138,7 @@ public class CarbonReaderBuilder {
    * Set the secret key for S3
    *
    * @param value the value of secret key
-   * @return CarbonWriterBuilder
+   * @return CarbonWriterBuilder object
    */
   public CarbonReaderBuilder setSecretKey(String value) {
     return setSecretKey(Constants.SECRET_KEY, value);
@@ -122,7 +149,7 @@ public class CarbonReaderBuilder {
    *
    * @param key   the string of endpoint for different S3 type,like: fs.s3a.endpoint
    * @param value the value of endpoint
-   * @return CarbonWriterBuilder
+   * @return CarbonWriterBuilder object
    */
   public CarbonReaderBuilder setEndPoint(String key, String value) {
     FileFactory.getConfiguration().set(key, value);
@@ -133,13 +160,20 @@ public class CarbonReaderBuilder {
    * Set the endpoint for S3
    *
    * @param value the value of endpoint
-   * @return CarbonWriterBuilder
+   * @return CarbonWriterBuilder object
    */
   public CarbonReaderBuilder setEndPoint(String value) {
-    FileFactory.getConfiguration().set(Constants.ENDPOINT, value);
-    return this;
+    return setEndPoint(Constants.ENDPOINT, value);
   }
 
+  /**
+   * Build CarbonReader
+   *
+   * @param <T>
+   * @return CarbonReader
+   * @throws IOException
+   * @throws InterruptedException
+   */
   public <T> CarbonReader<T> build() throws IOException, InterruptedException {
     CarbonTable table = CarbonTable.buildFromTablePath(tableName, tablePath, isTransactionalTable);
 

http://git-wip-us.apache.org/repos/asf/carbondata/blob/ddf3e859/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
----------------------------------------------------------------------
diff --git a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
index 2277ab0..e2dc8c2 100644
--- a/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
+++ b/store/sdk/src/main/java/org/apache/carbondata/sdk/file/CarbonWriterBuilder.java
@@ -120,10 +120,10 @@ public class CarbonWriterBuilder {
   /**
    * 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
+   * 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..
+   * If set to true, then writes the carbondata and carbonindex files
+   * in segment folder structure.
    * By default set to false.
    * @return updated CarbonWriterBuilder
    */
@@ -285,7 +285,7 @@ public class CarbonWriterBuilder {
   }
 
   /**
-   * To set the blocklet size of carbondata file
+   * To set the blocklet size of CarbonData file
    * @param blockletSize is blocklet size in MB
    * default value is 64 MB
    * @return updated CarbonWriterBuilder