Repository: carbondata-site
Updated Branches:
  refs/heads/asf-site bee563340 -> 6f8949f11


http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/6ad7599a/src/site/markdown/sdk-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/sdk-guide.md b/src/site/markdown/sdk-guide.md
index d786406..be42b3f 100644
--- a/src/site/markdown/sdk-guide.md
+++ b/src/site/markdown/sdk-guide.md
@@ -67,9 +67,9 @@ These SDK writer output contains just a carbondata and 
carbonindex files. No met
 
      CarbonProperties.getInstance().addProperty("enable.offheap.sort", 
enableOffheap);
  
-     CarbonWriterBuilder builder = CarbonWriter.builder().outputPath(path);
+     CarbonWriterBuilder builder = 
CarbonWriter.builder().outputPath(path).withCsvInput(schema);
  
-     CarbonWriter writer = builder.buildWriterForCSVInput(schema);
+     CarbonWriter writer = builder.build();
  
      int rows = 5;
      for (int i = 0; i < rows; i++) {
@@ -124,7 +124,7 @@ public class TestSdkAvro {
     try {
       CarbonWriter writer = CarbonWriter.builder()
           .outputPath(path)
-          .buildWriterForAvroInput(new 
org.apache.avro.Schema.Parser().parse(avroSchema));
+          .withAvroInput(new 
org.apache.avro.Schema.Parser().parse(avroSchema)).build();
 
       for (int i = 0; i < 100; i++) {
         writer.write(record);
@@ -164,10 +164,10 @@ public class TestSdkJson {
 
     Schema CarbonSchema = new Schema(fields);
 
-    CarbonWriterBuilder builder = CarbonWriter.builder().outputPath(path);
+    CarbonWriterBuilder builder = 
CarbonWriter.builder().outputPath(path).withJsonInput(CarbonSchema);
 
     // initialize json writer with carbon schema
-    CarbonWriter writer = builder.buildWriterForJsonInput(CarbonSchema);
+    CarbonWriter writer = builder.build();
     // one row of json Data as String
     String  JsonRow = "{\"name\":\"abcd\", \"age\":10}";
 
@@ -181,23 +181,34 @@ public class TestSdkJson {
 ```
 
 ## 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) |
+Each of SQL data types and Avro Data Types are mapped into data types of SDK. 
Following are the mapping:
+
+| SQL DataTypes | Avro DataTypes | Mapped SDK DataTypes |
+|---------------|----------------|----------------------|
+| BOOLEAN | BOOLEAN | DataTypes.BOOLEAN |
+| SMALLINT |  -  | DataTypes.SHORT |
+| INTEGER | INTEGER | DataTypes.INT |
+| BIGINT | LONG | DataTypes.LONG |
+| DOUBLE | DOUBLE | DataTypes.DOUBLE |
+| VARCHAR |  -  | DataTypes.STRING |
+| FLOAT | FLOAT | DataTypes.FLOAT |
+| BYTE |  -  | DataTypes.BYTE |
+| DATE | DATE | DataTypes.DATE |
+| TIMESTAMP |  -  | DataTypes.TIMESTAMP |
+| STRING | STRING | DataTypes.STRING |
+| DECIMAL | DECIMAL | DataTypes.createDecimalType(precision, scale) |
+| ARRAY | ARRAY | DataTypes.createArrayType(elementType) |
+| STRUCT | RECORD | DataTypes.createStructType(fields) |
+|  -  | ENUM | DataTypes.STRING |
+|  -  | UNION | DataTypes.createStructType(types) |
+|  -  | MAP | DataTypes.createMapType(keyType, valueType) |
+|  -  | TimeMillis | DataTypes.INT |
+|  -  | TimeMicros | DataTypes.LONG |
+|  -  | TimestampMillis | DataTypes.TIMESTAMP |
+|  -  | TimestampMicros | DataTypes.TIMESTAMP |
 
 **NOTE:**
- Carbon Supports below logical types of AVRO.
+ 1. Carbon Supports below logical types of AVRO.
  a. Date
     The date logical type represents a date within the calendar, with no 
reference to a particular time zone or time of day.
     A date logical type annotates an Avro int, where the int stores the number 
of days from the unix epoch, 1 January 1970 (ISO calendar). 
@@ -207,10 +218,22 @@ Each of SQL data types are mapped into data types of SDK. 
Following are the mapp
  c. Timestamp (microsecond precision)
     The timestamp-micros logical type represents an instant on the global 
timeline, independent of a particular time zone or calendar, with a precision 
of one microsecond.
     A timestamp-micros logical type annotates an Avro long, where the long 
stores the number of microseconds from the unix epoch, 1 January 1970 
00:00:00.000000 UTC.
+ d. Decimal
+    The decimal logical type represents an arbitrary-precision signed decimal 
number of the form unscaled × 10-scale.
+    A decimal logical type annotates Avro bytes or fixed types. The byte array 
must contain the two's-complement representation of the unscaled integer value 
in big-endian byte order. The scale is fixed, and is specified using an 
attribute.
+ e. Time (millisecond precision)
+    The time-millis logical type represents a time of day, with no reference 
to a particular calendar, time zone or date, with a precision of one 
millisecond.
+    A time-millis logical type annotates an Avro int, where the int stores the 
number of milliseconds after midnight, 00:00:00.000.
+ f. Time (microsecond precision)
+    The time-micros logical type represents a time of day, with no reference 
to a particular calendar, time zone or date, with a precision of one 
microsecond.
+    A time-micros logical type annotates an Avro long, where the long stores 
the number of microseconds after midnight, 00:00:00.000000.
+
     
     Currently the values of logical types are not validated by carbon. 
-    Expect that avro record passed by the user is already validated by avro 
record generator tools.   
-
+    Expect that avro record passed by the user is already validated by avro 
record generator tools.    
+ 2. If the string data is more than 32K in length, use withTableProperties() 
with "long_string_columns" property
+    or directly use DataTypes.VARCHAR if it is carbon schema.
+ 3. Avro Bytes, Fixed and Duration data types are not yet supported.
 ## Run SQL on files directly
 Instead of creating table and query it, you can also query that file directly 
with SQL.
 
@@ -234,20 +257,6 @@ 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.
@@ -306,16 +315,6 @@ 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.
@@ -339,7 +338,7 @@ public CarbonWriterBuilder taskNo(long taskNo);
 *                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"
@@ -359,99 +358,88 @@ public CarbonWriterBuilder withLoadOptions(Map<String, 
String> options);
 
 ```
 /**
- * To support the table properties for sdk writer
- *
- * @param options key,value pair of create table properties.
- * supported keys values are
- * a. blocksize -- [1-2048] values in MB. Default value is 1024
- * b. blockletsize -- values in MB. Default value is 64 MB
- * c. localDictionaryThreshold -- positive value, default is 10000
- * d. enableLocalDictionary -- true / false. Default is false
- * e. sortcolumns -- comma separated column. "c1,c2". Default all dimensions 
are sorted.
- *
- * @return updated CarbonWriterBuilder
- */
+* To support the table properties for sdk writer
+*
+* @param options key,value pair of create table properties.
+* supported keys values are
+* a. table_blocksize -- [1-2048] values in MB. Default value is 1024
+* b. table_blocklet_size -- values in MB. Default value is 64 MB
+* c. local_dictionary_threshold -- positive value, default is 10000
+* d. local_dictionary_enable -- true / false. Default is false
+* e. sort_columns -- comma separated column. "c1,c2". Default all dimensions 
are sorted.
+                     If empty string "" is passed. No columns are sorted
+* j. sort_scope -- "local_sort", "no_sort", "batch_sort". default value is 
"local_sort"
+* k. long_string_columns -- comma separated string columns which are more than 
32k length. 
+*                           default value is null.
+*
+* @return updated CarbonWriterBuilder
+*/
 public CarbonWriterBuilder withTableProperties(Map<String, String> options);
 ```
 
-
 ```
 /**
-* this writer is not thread safe, use buildThreadSafeWriterForCSVInput in 
multi thread environment
-* Build a {@link CarbonWriter}, which accepts row in CSV format object
-* @param schema carbon Schema object {org.apache.carbondata.sdk.file.Schema}
-* @return CSVCarbonWriter
-* @throws IOException
-* @throws InvalidLoadOptionException
+* To make sdk writer thread safe.
+*
+* @param numOfThreads should number of threads in which writer is called in 
multi-thread scenario
+*                     default sdk writer is not thread safe.
+*                     can use one writer instance in one thread only.
+* @return updated CarbonWriterBuilder
 */
-public CarbonWriter 
buildWriterForCSVInput(org.apache.carbondata.sdk.file.Schema schema) throws 
IOException, InvalidLoadOptionException;
+public CarbonWriterBuilder withThreadSafe(short numOfThreads);
 ```
 
 ```
 /**
-* Can use this writer in multi-thread instance.
-* Build a {@link CarbonWriter}, which accepts row in CSV format
-* @param schema carbon Schema object {org.apache.carbondata.sdk.file.Schema}
-* @param numOfThreads number of threads() in which .write will be called.      
        
-* @return CSVCarbonWriter
-* @throws IOException
-* @throws InvalidLoadOptionException
+* To support hadoop configuration
+*
+* @param conf hadoop configuration support, can set s3a AK,SK,end point and 
other conf with this
+* @return updated CarbonWriterBuilder
 */
-public CarbonWriter buildThreadSafeWriterForCSVInput(Schema schema, short 
numOfThreads)
-  throws IOException, InvalidLoadOptionException;
+public CarbonWriterBuilder withHadoopConf(Configuration conf)
 ```
 
-
-```  
+```
 /**
-* this writer is not thread safe, use buildThreadSafeWriterForAvroInput in 
multi thread environment
-* Build a {@link CarbonWriter}, which accepts Avro format object
-* @param avroSchema avro Schema object {org.apache.avro.Schema}
-* @return AvroCarbonWriter 
-* @throws IOException
-* @throws InvalidLoadOptionException
+* to build a {@link CarbonWriter}, which accepts row in CSV format
+*
+* @param schema carbon Schema object {org.apache.carbondata.sdk.file.Schema}
+* @return CarbonWriterBuilder
 */
-public CarbonWriter buildWriterForAvroInput(org.apache.avro.Schema schema) 
throws IOException, InvalidLoadOptionException;
+public CarbonWriterBuilder withCsvInput(Schema schema);
 ```
 
 ```
 /**
-* Can use this writer in multi-thread instance.
-* Build a {@link CarbonWriter}, which accepts Avro object
+* to build a {@link CarbonWriter}, which accepts Avro object
+*
 * @param avroSchema avro Schema object {org.apache.avro.Schema}
-* @param numOfThreads number of threads() in which .write will be called.
-* @return AvroCarbonWriter
-* @throws IOException
-* @throws InvalidLoadOptionException
+* @return CarbonWriterBuilder
 */
-public CarbonWriter buildThreadSafeWriterForAvroInput(org.apache.avro.Schema 
avroSchema, short numOfThreads)
-  throws IOException, InvalidLoadOptionException
+public CarbonWriterBuilder withAvroInput(org.apache.avro.Schema avroSchema);
 ```
 
-
 ```
 /**
-* this writer is not thread safe, use buildThreadSafeWriterForJsonInput in 
multi thread environment
-* Build a {@link CarbonWriter}, which accepts Json object
+* to build a {@link CarbonWriter}, which accepts Json object
+*
 * @param carbonSchema carbon Schema object
-* @return JsonCarbonWriter
-* @throws IOException
-* @throws InvalidLoadOptionException
+* @return CarbonWriterBuilder
 */
-public JsonCarbonWriter buildWriterForJsonInput(Schema carbonSchema);
+public CarbonWriterBuilder withJsonInput(Schema carbonSchema);
 ```
 
 ```
 /**
-* Can use this writer in multi-thread instance.
-* Build a {@link CarbonWriter}, which accepts Json object
-* @param carbonSchema carbon Schema object
-* @param numOfThreads number of threads() in which .write will be called.
-* @return JsonCarbonWriter
+* Build a {@link CarbonWriter}
+* This writer is not thread safe,
+* use withThreadSafe() configuration in multi thread environment
+* 
+* @return CarbonWriter {AvroCarbonWriter/CSVCarbonWriter/JsonCarbonWriter 
based on Input Type }
 * @throws IOException
 * @throws InvalidLoadOptionException
 */
-public JsonCarbonWriter buildThreadSafeWriterForJsonInput(Schema carbonSchema, 
short numOfThreads)
+public CarbonWriter build() throws IOException, InvalidLoadOptionException;
 ```
 
 ### Class org.apache.carbondata.sdk.file.CarbonWriter
@@ -462,7 +450,6 @@ public JsonCarbonWriter 
buildThreadSafeWriterForJsonInput(Schema carbonSchema, s
 *                      which is one row of data.
 * If CSVCarbonWriter, object is of type String[], which is one row of data
 * If JsonCarbonWriter, object is of type String, which is one row of json
-* Note: This API is not thread safe if writer is not built with number of 
threads argument.
 * @param object
 * @throws IOException
 */
@@ -636,19 +623,6 @@ Find example code at 
[CarbonReaderExample](https://github.com/apache/carbondata/
 ```
 
 ```
-  /**
-   * 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
   *
@@ -659,66 +633,13 @@ Find example code at 
[CarbonReaderExample](https://github.com/apache/carbondata/
 ```
 
 ```
-  /**
-   * 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);
+/**
+ * To support hadoop configuration
+ *
+ * @param conf hadoop configuration support, can set s3a AK,SK,end point and 
other conf with this
+ * @return updated CarbonReaderBuilder
+ */
+ public CarbonReaderBuilder withHadoopConf(Configuration conf);
 ```
 
 ```

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/6ad7599a/src/site/markdown/streaming-guide.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/streaming-guide.md 
b/src/site/markdown/streaming-guide.md
index 3b71662..714b07a 100644
--- a/src/site/markdown/streaming-guide.md
+++ b/src/site/markdown/streaming-guide.md
@@ -157,7 +157,7 @@ ALTER TABLE streaming_table SET 
TBLPROPERTIES('streaming'='true')
 At the begin of streaming ingestion, the system will try to acquire the table 
level lock of streaming.lock file. If the system isn't able to acquire the lock 
of this table, it will throw an InterruptedException.
 
 ## Create streaming segment
-The input data of streaming will be ingested into a segment of the CarbonData 
table, the status of this segment is streaming. CarbonData call it a streaming 
segment. The "tablestatus" file will record the segment status and data size. 
The user can use “SHOW SEGMENTS FOR TABLE tableName” to check segment 
status. 
+The streaming data will be ingested into a separate segment of carbondata 
table, this segment is termed as streaming segment. The status of this segment 
will be recorded as "streaming" in "tablestatus" file along with its data size. 
You can use "SHOW SEGMENTS FOR TABLE tableName" to check segment status. 
 
 After the streaming segment reaches the max size, CarbonData will change the 
segment status to "streaming finish" from "streaming", and create new 
"streaming" segment to continue to ingest streaming data.
 
@@ -304,8 +304,9 @@ Following example shows how to start a streaming ingest job
          | register TIMESTAMP,
          | updated TIMESTAMP
          |)
-         |STORED BY carbondata
+         |STORED AS carbondata
          |TBLPROPERTIES (
+         | 'streaming'='source',
          | 'format'='csv',
          | 'path'='$csvDataDir'
          |)
@@ -324,7 +325,7 @@ Following example shows how to start a streaming ingest job
          | register TIMESTAMP,
          | updated TIMESTAMP
          |)
-         |STORED BY carbondata
+         |STORED AS carbondata
          |TBLPROPERTIES (
          |  'streaming'='true'
          |)
@@ -351,7 +352,7 @@ Following example shows how to start a streaming ingest job
 
 In above example, two table is created: source and sink. The `source` table's 
format is `csv` and `sink` table format is `carbon`. Then a streaming job is 
created to stream data from source table to sink table.
 
-These two tables are normal carbon table, they can be queried independently.
+These two tables are normal carbon tables, they can be queried independently.
 
 
 
@@ -378,11 +379,14 @@ When this is issued, carbon will start a structured 
streaming job to do the stre
     name STRING,
     age INT
   )
-  STORED BY carbondata
+  STORED AS carbondata
   TBLPROPERTIES(
-    'format'='socket',
-    'host'='localhost',
-    'port'='8888'
+   'streaming'='source',
+   'format'='socket',
+   'host'='localhost',
+   'port'='8888',
+   'record_format'='csv', // can be csv or json, default is csv
+   'delimiter'='|'
   )
   ```
 
@@ -394,14 +398,31 @@ When this is issued, carbon will start a structured 
streaming job to do the stre
         .format("socket")
         .option("host", "localhost")
         .option("port", "8888")
+        .option("delimiter", "|")
   ```
 
 
 
 - The sink table should have a TBLPROPERTY `'streaming'` equal to `true`, 
indicating it is a streaming table.
 - In the given STMPROPERTIES, user must specify `'trigger'`, its value must be 
`ProcessingTime` (In future, other value will be supported). User should also 
specify interval value for the streaming job.
-- If the schema specifid in sink table is different from CTAS, the streaming 
job will fail
+- If the schema specified in sink table is different from CTAS, the streaming 
job will fail
 
+For Kafka data source, create the source table by:
+  ```SQL
+  CREATE TABLE source(
+    name STRING,
+    age INT
+  )
+  STORED AS carbondata
+  TBLPROPERTIES(
+   'streaming'='source',
+   'format'='kafka',
+   'kafka.bootstrap.servers'='kafkaserver:9092',
+   'subscribe'='test'
+   'record_format'='csv', // can be csv or json, default is csv
+   'delimiter'='|'
+  )
+  ```
 
 
 ##### STOP STREAM

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/6ad7599a/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 fee80f6..daf1acf 100644
--- a/src/site/markdown/supported-data-types-in-carbondata.md
+++ b/src/site/markdown/supported-data-types-in-carbondata.md
@@ -25,6 +25,10 @@
     * BIGINT
     * DOUBLE
     * DECIMAL
+    * FLOAT
+    * BYTE
+    
+    **NOTE**: Float and Bytes are only supported for SDK and FileFormat.
 
   * Date/Time Types
     * TIMESTAMP
@@ -41,6 +45,7 @@
   * Complex Types
     * arrays: ARRAY``<data_type>``
     * structs: STRUCT``<col_name : data_type COMMENT col_comment, ...>``
+    * maps: MAP``<primitive_type, data_type>``
     
     **NOTE**: Only 2 level complex type schema is supported for now.
 

http://git-wip-us.apache.org/repos/asf/carbondata-site/blob/6ad7599a/src/site/markdown/usecases.md
----------------------------------------------------------------------
diff --git a/src/site/markdown/usecases.md b/src/site/markdown/usecases.md
index 277c455..e8b98b5 100644
--- a/src/site/markdown/usecases.md
+++ b/src/site/markdown/usecases.md
@@ -39,7 +39,7 @@ These use cases can be broadly classified into below 
categories:
 
 ### Scenario
 
-User wants to analyse all the CHR(Call History Record) and MR(Measurement 
Records) of the mobile subscribers in order to identify the service failures 
within 10 secs.Also user wants to run machine learning models on the data to 
fairly estimate the reasons and time of probable failures and take action ahead 
to meet the SLA(Service Level Agreements) of VIP customers. 
+User wants to analyse all the CHR(Call History Record) and MR(Measurement 
Records) of the mobile subscribers in order to identify the service failures 
within 10 secs. Also user wants to run machine learning models on the data to 
fairly estimate the reasons and time of probable failures and take action ahead 
to meet the SLA(Service Level Agreements) of VIP customers. 
 
 ### Challenges
 
@@ -54,7 +54,7 @@ Setup a Hadoop + Spark + CarbonData cluster managed by YARN.
 
 Proposed the following configurations for CarbonData.(These tunings were 
proposed before CarbonData introduced SORT_COLUMNS parameter using which the 
sort order and schema order could be different.)
 
-Add the frequently used columns to the left of the table definition.Add it in 
the increasing order of cardinality.It was suggested to keep msisdn,imsi 
columns in the beginning of the schema.With latest CarbonData, SORT_COLUMNS 
needs to be configured msisdn,imsi in the beginning.
+Add the frequently used columns to the left of the table definition. Add it in 
the increasing order of cardinality. It was suggested to keep msisdn,imsi 
columns in the beginning of the schema. With latest CarbonData, SORT_COLUMNS 
needs to be configured msisdn,imsi in the beginning.
 
 Add timestamp column to the right of the schema as it is naturally increasing.
 
@@ -71,7 +71,7 @@ Apart from these, the following CarbonData configuration was 
suggested to be con
 | Data Loading | carbon.sort.size                        | 100000 | Number of 
records to sort at a time.More number of records configured will lead to 
increased memory foot print |
 | Data Loading | table_blocksize                         | 256  | To 
efficiently schedule multiple tasks during query |
 | Data Loading | carbon.sort.intermediate.files.limit    | 100    | Increased 
to 100 as number of cores are more.Can perform merging in backgorund.If less 
number of files to merge, sort threads would be idle |
-| Data Loading | carbon.use.local.dir                    | TRUE   | yarn 
application directory will be usually on a single disk.YARN would be configured 
with multiple disks to be used as temp or to assign randomly to 
applications.Using the yarn temp directory will allow carbon to use multiple 
disks and improve IO performance |
+| Data Loading | carbon.use.local.dir                    | TRUE   | yarn 
application directory will be usually on a single disk.YARN would be configured 
with multiple disks to be used as temp or to assign randomly to applications. 
Using the yarn temp directory will allow carbon to use multiple disks and 
improve IO performance |
 | Data Loading | carbon.use.multiple.temp.dir            | TRUE   | multiple 
disks to write sort files will lead to better IO and reduce the IO bottleneck |
 | Compaction | carbon.compaction.level.threshold       | 6,6    | Since 
frequent small loads, compacting more segments will give better query results |
 | Compaction | carbon.enable.auto.load.merge           | true   | Since data 
loading is small,auto compacting keeps the number of segments less and also 
compaction can complete in  time |
@@ -94,7 +94,7 @@ Apart from these, the following CarbonData configuration was 
suggested to be con
 
 ### Scenario
 
-User wants to analyse the person/vehicle movement and behavior during a 
certain time period.This output data needs to be joined with a external table 
for Human details extraction.The query will be run with different time period 
as filter to identify potential behavior mismatch.
+User wants to analyse the person/vehicle movement and behavior during a 
certain time period. This output data needs to be joined with a external table 
for Human details extraction. The query will be run with different time period 
as filter to identify potential behavior mismatch.
 
 ### Challenges
 
@@ -119,24 +119,24 @@ Use all columns are no-dictionary as the cardinality is 
high.
 | Configuration for | Parameter                               | Value          
         | Description |
 | ------------------| --------------------------------------- | 
----------------------- | ------------------|
 | Data Loading | carbon.graph.rowset.size                | 100000              
    | Based on the size of each row, this determines the memory required during 
data loading.Higher value leads to increased memory foot print |
-| Data Loading | enable.unsafe.sort                      | TRUE                
    | Temporary data generated during sort is huge which causes GC 
bottlenecks.Using unsafe reduces the pressure on GC |
-| Data Loading | enable.offheap.sort                     | TRUE                
    | Temporary data generated during sort is huge which causes GC 
bottlenecks.Using offheap reduces the pressure on GC.offheap can be accessed 
through java unsafe.hence enable.unsafe.sort needs to be true |
+| Data Loading | enable.unsafe.sort                      | TRUE                
    | Temporary data generated during sort is huge which causes GC bottlenecks. 
Using unsafe reduces the pressure on GC |
+| Data Loading | enable.offheap.sort                     | TRUE                
    | Temporary data generated during sort is huge which causes GC bottlenecks. 
Using offheap reduces the pressure on GC.offheap can be accessed through java 
unsafe.hence enable.unsafe.sort needs to be true |
 | Data Loading | offheap.sort.chunk.size.in.mb           | 128                 
    | Size of memory to allocate for sorting.Can increase this based on the 
memory available |
 | Data Loading | carbon.number.of.cores.while.loading    | 12                  
    | Higher cores can improve data loading speed |
 | Data Loading | carbon.sort.size                        | 100000              
    | Number of records to sort at a time.More number of records configured 
will lead to increased memory foot print |
-| Data Loading | table_blocksize                         | 512                 
    | To efficiently schedule multiple tasks during query.This size depends on 
data scenario.If data is such that the filters would select less number of 
blocklets to scan, keeping higher number works well.If the number blocklets to 
scan is more, better to reduce the size as more tasks can be scheduled in 
parallel. |
+| Data Loading | table_blocksize                         | 512                 
    | To efficiently schedule multiple tasks during query. This size depends on 
data scenario.If data is such that the filters would select less number of 
blocklets to scan, keeping higher number works well.If the number blocklets to 
scan is more, better to reduce the size as more tasks can be scheduled in 
parallel. |
 | Data Loading | carbon.sort.intermediate.files.limit    | 100                 
    | Increased to 100 as number of cores are more.Can perform merging in 
backgorund.If less number of files to merge, sort threads would be idle |
-| Data Loading | carbon.use.local.dir                    | TRUE                
    | yarn application directory will be usually on a single disk.YARN would be 
configured with multiple disks to be used as temp or to assign randomly to 
applications.Using the yarn temp directory will allow carbon to use multiple 
disks and improve IO performance |
+| Data Loading | carbon.use.local.dir                    | TRUE                
    | yarn application directory will be usually on a single disk.YARN would be 
configured with multiple disks to be used as temp or to assign randomly to 
applications. Using the yarn temp directory will allow carbon to use multiple 
disks and improve IO performance |
 | Data Loading | carbon.use.multiple.temp.dir            | TRUE                
    | multiple disks to write sort files will lead to better IO and reduce the 
IO bottleneck |
-| Data Loading | sort.inmemory.size.in.mb                | 92160 | Memory 
allocated to do inmemory sorting.When more memory is available in the node, 
configuring this will retain more sort blocks in memory so that the merge sort 
is faster due to no/very less IO |
+| Data Loading | sort.inmemory.size.in.mb                | 92160 | Memory 
allocated to do inmemory sorting. When more memory is available in the node, 
configuring this will retain more sort blocks in memory so that the merge sort 
is faster due to no/very less IO |
 | Compaction | carbon.major.compaction.size            | 921600                
  | Sum of several loads to combine into single segment |
 | Compaction | carbon.number.of.cores.while.compacting | 12                    
  | Higher number of cores can improve the compaction speed.Data size is 
huge.Compaction need to use more threads to speed up the process |
-| Compaction | carbon.enable.auto.load.merge           | FALSE                 
  | Doing auto minor compaction is costly process as data size is huge.Perform 
manual compaction when  the cluster is less loaded |
+| Compaction | carbon.enable.auto.load.merge           | FALSE                 
  | Doing auto minor compaction is costly process as data size is huge.Perform 
manual compaction when the cluster is less loaded |
 | Query | carbon.enable.vector.reader             | true                    | 
To fetch results faster, supporting spark vector processing will speed up the 
query |
-| Query | enable.unsafe.in.query.procressing      | true                    | 
Data that needs to be scanned in huge which in turn generates more short lived 
Java objects.This cause pressure of GC.using unsafe and offheap will reduce the 
GC overhead |
-| Query | use.offheap.in.query.processing         | true                    | 
Data that needs to be scanned in huge which in turn generates more short lived 
Java objects.This cause pressure of GC.using unsafe and offheap will reduce the 
GC overhead.offheap can be accessed through java unsafe.hence 
enable.unsafe.in.query.procressing needs to be true |
+| Query | enable.unsafe.in.query.procressing      | true                    | 
Data that needs to be scanned in huge which in turn generates more short lived 
Java objects. This cause pressure of GC.using unsafe and offheap will reduce 
the GC overhead |
+| Query | use.offheap.in.query.processing         | true                    | 
Data that needs to be scanned in huge which in turn generates more short lived 
Java objects. This cause pressure of GC.using unsafe and offheap will reduce 
the GC overhead.offheap can be accessed through java unsafe.hence 
enable.unsafe.in.query.procressing needs to be true |
 | Query | enable.unsafe.columnpage                | TRUE                    | 
Keep the column pages in offheap memory so that the memory overhead due to java 
object is less and also reduces GC pressure. |
-| Query | carbon.unsafe.working.memory.in.mb      | 10240                   | 
Amount of memory to use for offheap operations.Can increase this memory based 
on the data size |
+| Query | carbon.unsafe.working.memory.in.mb      | 10240                   | 
Amount of memory to use for offheap operations, you can increase this memory 
based on the data size |
 
 
 
@@ -177,7 +177,7 @@ Concurrent queries can be more due to the BI dashboard
 - Create pre-aggregate tables for non timestamp based group by queries
 - For queries containing group by date, create timeseries based 
Datamap(pre-aggregate) tables so that the data is rolled up during creation and 
fetch is faster
 - Reduce the Spark shuffle partitions.(In our configuration on 14 node 
cluster, it was reduced to 35 from default of 200)
-- Enable global dictionary for columns which have less 
cardinalities.Aggregation can be done on encoded data, there by improving the 
performance
+- Enable global dictionary for columns which have less cardinalities. 
Aggregation can be done on encoded data, there by improving the performance
 - For columns whose cardinality is high,enable the local dictionary so that 
store size is less and can take dictionary benefit for scan
 
 ## Handling near realtime data ingestion scenario
@@ -188,14 +188,14 @@ Need to support storing of continously arriving data and 
make it available immed
 
 ### Challenges
 
-When the data ingestion is near real time and the data needs to be available 
for query immediately, usual scenario is to do data loading in micro 
batches.But this causes the problem of generating many small files.This poses 
two problems:
+When the data ingestion is near real time and the data needs to be available 
for query immediately, usual scenario is to do data loading in micro 
batches.But this causes the problem of generating many small files. This poses 
two problems:
 
 1. Small file handling in HDFS is inefficient
 2. CarbonData will suffer in query performance as all the small files will 
have to be queried when filter is on non time column
 
 CarbonData will suffer in query performance as all the small files will have 
to be queried when filter is on non time column.
 
-Since data is continouly arriving, allocating resources for compaction might 
not be feasible.
+Since data is continously arriving, allocating resources for compaction might 
not be feasible.
 
 ### Goal
 

Reply via email to