You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by kunal642 <gi...@git.apache.org> on 2018/07/29 16:16:47 UTC

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] disable preagg datamap on d...

GitHub user kunal642 opened a pull request:

    https://github.com/apache/carbondata/pull/2576

    [CARBONDATA-2795] disable preagg datamap on dataload for s3 store

    Be sure to do all of the following checklist to help us incorporate 
    your contribution quickly and easily:
    
     - [ ] Any interfaces changed?
     
     - [ ] Any backward compatibility impacted?
     
     - [ ] Document update required?
    
     - [ ] Testing done
            Please provide details on 
            - Whether new unit test cases have been added or why no new tests are required?
            - How it is tested? Please attach test report.
            - Is it a performance related change? Please attach the performance test report.
            - Any additional information to help reviewers in testing this change.
           
     - [ ] For large changes, please consider breaking it into sub-tasks under an umbrella JIRA. 
    


You can merge this pull request into a Git repository by running:

    $ git pull https://github.com/kunal642/carbondata disable_preagg

Alternatively you can review and apply these changes as the patch at:

    https://github.com/apache/carbondata/pull/2576.patch

To close this pull request, make a commit to your master/trunk branch
with (at least) the following in the commit message:

    This closes #2576
    
----
commit ee154f1e94e8ada5fcba08cda3534e9d80ee506a
Author: kunal642 <ku...@...>
Date:   2018-07-29T16:14:22Z

    disable preagg datamap on dataload for s3 store

----


---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by kunal642 <gi...@git.apache.org>.
Github user kunal642 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207250066
  
    --- Diff: docs/configuration-parameters.md ---
    @@ -106,7 +106,10 @@ This section provides the details of all the configurations required for CarbonD
     |---------------------------------------------|--------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
     | carbon.sort.file.write.buffer.size | 16384 | File write buffer size used during sorting. Minimum allowed buffer size is 10240 byte and Maximum allowed buffer size is 10485760 byte. |
     | carbon.lock.type | LOCALLOCK | This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently. - HDFSLOCK: Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData spark applications are launched and no ZooKeeper is running on cluster and HDFS supports file based locking. |
    -| carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3.
    +| carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to 
    --- End diff --
    
    added description


---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by kunal642 <gi...@git.apache.org>.
Github user kunal642 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207250192
  
    --- Diff: docs/s3-guide.md ---
    @@ -0,0 +1,64 @@
    +<!--
    +    Licensed to the Apache Software Foundation (ASF) under one or more 
    +    contributor license agreements.  See the NOTICE file distributed with
    +    this work for additional information regarding copyright ownership. 
    +    The ASF licenses this file to you under the Apache License, Version 2.0
    +    (the "License"); you may not use this file except in compliance with 
    +    the License.  You may obtain a copy of the License at
    +
    +      http://www.apache.org/licenses/LICENSE-2.0
    +
    +    Unless required by applicable law or agreed to in writing, software 
    +    distributed under the License is distributed on an "AS IS" BASIS, 
    +    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +    See the License for the specific language governing permissions and 
    +    limitations under the License.
    +-->
    +
    +#S3 Guide (Alpha Feature 1.4.1)
    +S3 is an Object Storage API on cloud, it is recommended for storing large data files. You can use 
    +this feature if you want to store data on Amazon cloud or Huawei cloud(OBS).
    +Since the data is stored on to cloud there are no restrictions on the size of data and the data can be accessed from anywhere at any time.
    +Carbondata can support any Object Storage that conforms to Amazon S3 API.
    --- End diff --
    
    merged


---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by sraghunandan <gi...@git.apache.org>.
Github user sraghunandan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207071826
  
    --- Diff: docs/configuration-parameters.md ---
    @@ -106,7 +106,10 @@ This section provides the details of all the configurations required for CarbonD
     |---------------------------------------------|--------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
     | carbon.sort.file.write.buffer.size | 16384 | File write buffer size used during sorting. Minimum allowed buffer size is 10240 byte and Maximum allowed buffer size is 10485760 byte. |
     | carbon.lock.type | LOCALLOCK | This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently. - HDFSLOCK: Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData spark applications are launched and no ZooKeeper is running on cluster and HDFS supports file based locking. |
    -| carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3.
    +| carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to 
    --- End diff --
    
    add a brief description as to why locks are used in carbondata.what is TABLEPATH ?


---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by asfgit <gi...@git.apache.org>.
Github user asfgit closed the pull request at:

    https://github.com/apache/carbondata/pull/2576


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/6154/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/7733/



---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by chenliang613 <gi...@git.apache.org>.
Github user chenliang613 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r206480055
  
    --- Diff: docs/datamap/preaggregate-datamap-guide.md ---
    @@ -7,6 +24,7 @@
     * [Querying Data](#querying-data)
     * [Compaction](#compacting-pre-aggregate-tables)
     * [Data Management](#data-management-with-pre-aggregate-tables)
    +* [Limitations](#Limitations)
    --- End diff --
    
    Why need to add this item


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/6116/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/6115/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6449/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6459/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by chenliang613 <gi...@git.apache.org>.
Github user chenliang613 commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    retest this please


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by kunal642 <gi...@git.apache.org>.
Github user kunal642 commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    @sraghunandan @sgururajshetty @chenliang613 Please review


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6360/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/6121/



---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by chenliang613 <gi...@git.apache.org>.
Github user chenliang613 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r206481369
  
    --- Diff: docs/s3-guide.md ---
    @@ -0,0 +1,63 @@
    +<!--
    +    Licensed to the Apache Software Foundation (ASF) under one or more 
    +    contributor license agreements.  See the NOTICE file distributed with
    +    this work for additional information regarding copyright ownership. 
    +    The ASF licenses this file to you under the Apache License, Version 2.0
    +    (the "License"); you may not use this file except in compliance with 
    +    the License.  You may obtain a copy of the License at
    +
    +      http://www.apache.org/licenses/LICENSE-2.0
    +
    +    Unless required by applicable law or agreed to in writing, software 
    +    distributed under the License is distributed on an "AS IS" BASIS, 
    +    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +    See the License for the specific language governing permissions and 
    +    limitations under the License.
    +-->
    +
    +#S3 Guide (Alpha Feature 1.4.1)
    +Amazon S3 is a cloud storage service that is recommended for storing large data files. You can 
    --- End diff --
    
    Suggest changing to : 
    
    S3 is an object storage API on cloud,it is recommended for storing large data files. You can use this feature if you want to store data on amazon cloud or huawei cloud(obs). Since the data is stored on cloud 
    storage there are no restrictions on the size of data and the data can be accessed from anywhere at any time.
    Carbondata can support any Object storage that conforms to Amazon S3 API.


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6455/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/6080/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6404/



---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by kunal642 <gi...@git.apache.org>.
Github user kunal642 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207250096
  
    --- Diff: docs/configuration-parameters.md ---
    @@ -106,7 +106,10 @@ This section provides the details of all the configurations required for CarbonD
     |---------------------------------------------|--------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
     | carbon.sort.file.write.buffer.size | 16384 | File write buffer size used during sorting. Minimum allowed buffer size is 10240 byte and Maximum allowed buffer size is 10485760 byte. |
     | carbon.lock.type | LOCALLOCK | This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently. - HDFSLOCK: Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData spark applications are launched and no ZooKeeper is running on cluster and HDFS supports file based locking. |
    -| carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3.
    +| carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to 
    +be created. Recommended to configure HDFS lock path(to this property) in case of S3 file system 
    +as locking is not feasible on S3. 
    +**Note:** If this property is not set to HDFS location for S3 store, then there is a possibility of data corruption. 
    --- End diff --
    
    done


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by chenliang613 <gi...@git.apache.org>.
Github user chenliang613 commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    retest this please


---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by kunal642 <gi...@git.apache.org>.
Github user kunal642 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207249485
  
    --- Diff: docs/configuration-parameters.md ---
    @@ -106,7 +106,12 @@ This section provides the details of all the configurations required for CarbonD
     |---------------------------------------------|--------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
     | carbon.sort.file.write.buffer.size | 16384 | File write buffer size used during sorting. Minimum allowed buffer size is 10240 byte and Maximum allowed buffer size is 10485760 byte. |
     | carbon.lock.type | LOCALLOCK | This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently. - HDFSLOCK: Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData spark applications are launched and no ZooKeeper is running on cluster and HDFS supports file based locking. |
    -| carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3.
    +| carbon.lock.path | TABLEPATH | Locks on the files are used to prevent concurrent operation from modifying the same files. This 
    +configuration specifies the path where lock files have to be created. Recommended to configure 
    +HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3. 
    +**Note:** If this property is not set to HDFS location for S3 store, then there is a possibility 
    +of data corruption because multiple data manipulation calls might try to update the status file 
    +and as lock is not acquired before updation data might get overwritten.
    --- End diff --
    
    added


---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by chenliang613 <gi...@git.apache.org>.
Github user chenliang613 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r206481821
  
    --- Diff: docs/s3-guide.md ---
    @@ -0,0 +1,63 @@
    +<!--
    +    Licensed to the Apache Software Foundation (ASF) under one or more 
    +    contributor license agreements.  See the NOTICE file distributed with
    +    this work for additional information regarding copyright ownership. 
    +    The ASF licenses this file to you under the Apache License, Version 2.0
    +    (the "License"); you may not use this file except in compliance with 
    +    the License.  You may obtain a copy of the License at
    +
    +      http://www.apache.org/licenses/LICENSE-2.0
    +
    +    Unless required by applicable law or agreed to in writing, software 
    +    distributed under the License is distributed on an "AS IS" BASIS, 
    +    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +    See the License for the specific language governing permissions and 
    +    limitations under the License.
    +-->
    +
    +#S3 Guide (Alpha Feature 1.4.1)
    +Amazon S3 is a cloud storage service that is recommended for storing large data files. You can 
    +use this feature if you want to store data on amazon cloud. Since the data is stored on to cloud 
    +storage there are no restrictions on the size of data and the data can be accessed from anywhere at any time.
    +Carbon can support any Object store that conforms to Amazon S3 API. 
    +
    +#Writing to Object Store
    +To store carbondata files on to Object Store location, you need to set `carbon
    +.storelocation` property to Object Store path in CarbonProperties file. For example, carbon
    +.storelocation=s3a://mybucket/carbonstore. By setting this property, all the tables will be created on the specified Object Store path.
    +
    +If your existing store is HDFS, and you want to store specific tables on S3 location, then `location` parameter has to be set during create 
    +table. 
    +For example:
    +
    +```
    +CREATE TABLE IF NOT EXISTS db1.table1(col1 string, col2 int) STORED AS carbondata LOCATION 's3a://mybucket/carbonstore'
    +``` 
    +
    +For more details on create table, Refer [data-management-on-carbondata](https://github.com/apache/carbondata/blob/master/docs/data-management-on-carbondata.md#create-table)
    +
    +#Authentication
    +You need to set authentication properties to store the carbondata files on to S3 location. For 
    +more details on authentication properties, refer 
    +[hadoop authentication document](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#Authentication_properties)
    +
    +Another way of setting the authentication parameters is as follows:
    +
    +```
    +         SparkSession
    +         .builder()
    +         .master(masterURL)
    +         .appName("S3Example")
    +         .config("spark.driver.host", "localhost")
    +         .config("spark.hadoop.fs.s3a.access.key", "1111")
    +         .config("spark.hadoop.fs.s3a.secret.key", "2222")
    +         .config("spark.hadoop.fs.s3a.endpoint", "1.1.1.1")
    +         .getOrCreateCarbonSession()
    +```
    +
    +#Recommendations
    +1. Object stores like S3 does not support file leasing mechanism(supported by HDFS) that is 
    +required to take locks which ensure consistency between concurrent operations therefore, it is 
    +recommended to set the configurable lock path property([carbon.lock.path](https://github.com/apache/carbondata/blob/master/docs/configuration-parameters.md#miscellaneous-configuration))
    + to a HDFS directory.
    +2. As Object stores are eventual consistent meaning that any put request can take some time to reflect when trying to list objects from that bucket therefore concurrent queries are not supported. 
    --- End diff --
    
    Changes to : Object Storage


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/7775/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/6131/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by kunal642 <gi...@git.apache.org>.
Github user kunal642 commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    retest this please


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] disable preagg datamap on dataload...

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/6056/



---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by sraghunandan <gi...@git.apache.org>.
Github user sraghunandan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207073807
  
    --- Diff: docs/s3-guide.md ---
    @@ -0,0 +1,64 @@
    +<!--
    +    Licensed to the Apache Software Foundation (ASF) under one or more 
    +    contributor license agreements.  See the NOTICE file distributed with
    +    this work for additional information regarding copyright ownership. 
    +    The ASF licenses this file to you under the Apache License, Version 2.0
    +    (the "License"); you may not use this file except in compliance with 
    +    the License.  You may obtain a copy of the License at
    +
    +      http://www.apache.org/licenses/LICENSE-2.0
    +
    +    Unless required by applicable law or agreed to in writing, software 
    +    distributed under the License is distributed on an "AS IS" BASIS, 
    +    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +    See the License for the specific language governing permissions and 
    +    limitations under the License.
    +-->
    +
    +#S3 Guide (Alpha Feature 1.4.1)
    +S3 is an Object Storage API on cloud, it is recommended for storing large data files. You can use 
    +this feature if you want to store data on Amazon cloud or Huawei cloud(OBS).
    +Since the data is stored on to cloud there are no restrictions on the size of data and the data can be accessed from anywhere at any time.
    +Carbondata can support any Object Storage that conforms to Amazon S3 API.
    +
    +#Writing to Object Storage
    +To store carbondata files on to Object Store location, you need to set `carbon
    +.storelocation` property to Object Store path in CarbonProperties file. For example, carbon
    +.storelocation=s3a://mybucket/carbonstore. By setting this property, all the tables will be created on the specified Object Store path.
    +
    +If your existing store is HDFS, and you want to store specific tables on S3 location, then `location` parameter has to be set during create 
    --- End diff --
    
    If you don't wish to change the existing store location and would wish to store only specific tables onto S3,it can be done by setting the 'location' option parameter in the create table ddl command


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] disable preagg datamap on dataload...

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/7593/



---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by sraghunandan <gi...@git.apache.org>.
Github user sraghunandan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207074600
  
    --- Diff: docs/s3-guide.md ---
    @@ -0,0 +1,64 @@
    +<!--
    +    Licensed to the Apache Software Foundation (ASF) under one or more 
    +    contributor license agreements.  See the NOTICE file distributed with
    +    this work for additional information regarding copyright ownership. 
    +    The ASF licenses this file to you under the Apache License, Version 2.0
    +    (the "License"); you may not use this file except in compliance with 
    +    the License.  You may obtain a copy of the License at
    +
    +      http://www.apache.org/licenses/LICENSE-2.0
    +
    +    Unless required by applicable law or agreed to in writing, software 
    +    distributed under the License is distributed on an "AS IS" BASIS, 
    +    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +    See the License for the specific language governing permissions and 
    +    limitations under the License.
    +-->
    +
    +#S3 Guide (Alpha Feature 1.4.1)
    +S3 is an Object Storage API on cloud, it is recommended for storing large data files. You can use 
    +this feature if you want to store data on Amazon cloud or Huawei cloud(OBS).
    +Since the data is stored on to cloud there are no restrictions on the size of data and the data can be accessed from anywhere at any time.
    +Carbondata can support any Object Storage that conforms to Amazon S3 API.
    +
    +#Writing to Object Storage
    +To store carbondata files on to Object Store location, you need to set `carbon
    +.storelocation` property to Object Store path in CarbonProperties file. For example, carbon
    +.storelocation=s3a://mybucket/carbonstore. By setting this property, all the tables will be created on the specified Object Store path.
    +
    +If your existing store is HDFS, and you want to store specific tables on S3 location, then `location` parameter has to be set during create 
    +table. 
    +For example:
    +
    +```
    +CREATE TABLE IF NOT EXISTS db1.table1(col1 string, col2 int) STORED AS carbondata LOCATION 's3a://mybucket/carbonstore'
    +``` 
    +
    +For more details on create table, Refer [data-management-on-carbondata](https://github.com/apache/carbondata/blob/master/docs/data-management-on-carbondata.md#create-table)
    +
    +#Authentication
    +You need to set authentication properties to store the carbondata files on to S3 location. For 
    +more details on authentication properties, refer 
    +[hadoop authentication document](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#Authentication_properties)
    +
    +Another way of setting the authentication parameters is as follows:
    +
    +```
    +         SparkSession
    +         .builder()
    +         .master(masterURL)
    +         .appName("S3Example")
    +         .config("spark.driver.host", "localhost")
    +         .config("spark.hadoop.fs.s3a.access.key", "1111")
    +         .config("spark.hadoop.fs.s3a.secret.key", "2222")
    +         .config("spark.hadoop.fs.s3a.endpoint", "1.1.1.1")
    +         .getOrCreateCarbonSession()
    +```
    +
    +#Recommendations
    +1. Object Storage like S3 does not support file leasing mechanism(supported by HDFS) that is 
    +required to take locks which ensure consistency between concurrent operations therefore, it is 
    +recommended to set the configurable lock path property([carbon.lock.path](https://github.com/apache/carbondata/blob/master/docs/configuration-parameters.md#miscellaneous-configuration))
    + to a HDFS directory.
    +2. As Object Storage are eventual consistent meaning that any put request can take some time to 
    --- End diff --
    
    Concurrent data manipulation operations are not supported. object stores follow eventual consistency semantics,ie.,any put request might take some time to reflect when trying to list.This behaviour causes not to ensure the data read is always consistent or latest.


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by sgururajshetty <gi...@git.apache.org>.
Github user sgururajshetty commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    LGTM


---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by sgururajshetty <gi...@git.apache.org>.
Github user sgururajshetty commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207223686
  
    --- Diff: docs/configuration-parameters.md ---
    @@ -106,7 +106,12 @@ This section provides the details of all the configurations required for CarbonD
     |---------------------------------------------|--------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
     | carbon.sort.file.write.buffer.size | 16384 | File write buffer size used during sorting. Minimum allowed buffer size is 10240 byte and Maximum allowed buffer size is 10485760 byte. |
     | carbon.lock.type | LOCALLOCK | This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently. - HDFSLOCK: Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData spark applications are launched and no ZooKeeper is running on cluster and HDFS supports file based locking. |
    -| carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3.
    +| carbon.lock.path | TABLEPATH | Locks on the files are used to prevent concurrent operation from modifying the same files. This 
    +configuration specifies the path where lock files have to be created. Recommended to configure 
    +HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3. 
    +**Note:** If this property is not set to HDFS location for S3 store, then there is a possibility 
    +of data corruption because multiple data manipulation calls might try to update the status file 
    +and as lock is not acquired before updation data might get overwritten.
    --- End diff --
    
    since it is table, end the line with a pipline |



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6499/



---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by sraghunandan <gi...@git.apache.org>.
Github user sraghunandan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207072154
  
    --- Diff: docs/data-management-on-carbondata.md ---
    @@ -730,6 +736,8 @@ Users can specify which columns to include and exclude for local dictionary gene
       * If the IGNORE option is used, then bad records are neither loaded nor written to the separate CSV file.
       * In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails.
       * The maximum number of characters per column is 32000. If there are more than 32000 characters in a column, data loading will fail.
    +  * Since Bad Records Path can be specified in both create, load and carbon properties. 
    --- End diff --
    
    entire sentence to be reformed. not a grammatically correct statement


---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by sraghunandan <gi...@git.apache.org>.
Github user sraghunandan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207072493
  
    --- Diff: docs/s3-guide.md ---
    @@ -0,0 +1,64 @@
    +<!--
    +    Licensed to the Apache Software Foundation (ASF) under one or more 
    +    contributor license agreements.  See the NOTICE file distributed with
    +    this work for additional information regarding copyright ownership. 
    +    The ASF licenses this file to you under the Apache License, Version 2.0
    +    (the "License"); you may not use this file except in compliance with 
    +    the License.  You may obtain a copy of the License at
    +
    +      http://www.apache.org/licenses/LICENSE-2.0
    +
    +    Unless required by applicable law or agreed to in writing, software 
    +    distributed under the License is distributed on an "AS IS" BASIS, 
    +    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +    See the License for the specific language governing permissions and 
    +    limitations under the License.
    +-->
    +
    +#S3 Guide (Alpha Feature 1.4.1)
    +S3 is an Object Storage API on cloud, it is recommended for storing large data files. You can use 
    +this feature if you want to store data on Amazon cloud or Huawei cloud(OBS).
    +Since the data is stored on to cloud there are no restrictions on the size of data and the data can be accessed from anywhere at any time.
    +Carbondata can support any Object Storage that conforms to Amazon S3 API.
    --- End diff --
    
    This sentence can be merged with the above sentence "You can use this feature if you want to store data ...."


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/7669/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6467/



---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by sraghunandan <gi...@git.apache.org>.
Github user sraghunandan commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207071907
  
    --- Diff: docs/configuration-parameters.md ---
    @@ -106,7 +106,10 @@ This section provides the details of all the configurations required for CarbonD
     |---------------------------------------------|--------------------------|---------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------------|
     | carbon.sort.file.write.buffer.size | 16384 | File write buffer size used during sorting. Minimum allowed buffer size is 10240 byte and Maximum allowed buffer size is 10485760 byte. |
     | carbon.lock.type | LOCALLOCK | This configuration specifies the type of lock to be acquired during concurrent operations on table. There are following types of lock implementation: - LOCALLOCK: Lock is created on local file system as file. This lock is useful when only one spark driver (thrift server) runs on a machine and no other CarbonData spark application is launched concurrently. - HDFSLOCK: Lock is created on HDFS file system as file. This lock is useful when multiple CarbonData spark applications are launched and no ZooKeeper is running on cluster and HDFS supports file based locking. |
    -| carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to be created. Recommended to configure zookeeper lock type or configure HDFS lock path(to this property) in case of S3 file system as locking is not feasible on S3.
    +| carbon.lock.path | TABLEPATH | This configuration specifies the path where lock files have to 
    +be created. Recommended to configure HDFS lock path(to this property) in case of S3 file system 
    +as locking is not feasible on S3. 
    +**Note:** If this property is not set to HDFS location for S3 store, then there is a possibility of data corruption. 
    --- End diff --
    
    can add a brief sentence as to why corruption might happen


---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by kunal642 <gi...@git.apache.org>.
Github user kunal642 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207249849
  
    --- Diff: docs/datamap/preaggregate-datamap-guide.md ---
    @@ -7,6 +24,7 @@
     * [Querying Data](#querying-data)
     * [Compaction](#compacting-pre-aggregate-tables)
     * [Data Management](#data-management-with-pre-aggregate-tables)
    +* [Limitations](#Limitations)
    --- End diff --
    
    removed


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6500/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/6149/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/7679/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/7742/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/7614/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/7729/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6492/



---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by KanakaKumar <gi...@git.apache.org>.
Github user KanakaKumar commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r206515601
  
    --- Diff: docs/data-management-on-carbondata.md ---
    @@ -730,6 +736,8 @@ Users can specify which columns to include and exclude for local dictionary gene
       * If the IGNORE option is used, then bad records are neither loaded nor written to the separate CSV file.
       * In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails.
       * The maximum number of characters per column is 32000. If there are more than 32000 characters in a column, data loading will fail.
    +  * Since Bad Records Path can be specified in both create, load and carbon properties. 
    --- End diff --
    
    "both" does not suite in this statement. Please rewrite.


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/6158/



---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by kunal642 <gi...@git.apache.org>.
Github user kunal642 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207249941
  
    --- Diff: docs/s3-guide.md ---
    @@ -0,0 +1,63 @@
    +<!--
    +    Licensed to the Apache Software Foundation (ASF) under one or more 
    +    contributor license agreements.  See the NOTICE file distributed with
    +    this work for additional information regarding copyright ownership. 
    +    The ASF licenses this file to you under the Apache License, Version 2.0
    +    (the "License"); you may not use this file except in compliance with 
    +    the License.  You may obtain a copy of the License at
    +
    +      http://www.apache.org/licenses/LICENSE-2.0
    +
    +    Unless required by applicable law or agreed to in writing, software 
    +    distributed under the License is distributed on an "AS IS" BASIS, 
    +    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +    See the License for the specific language governing permissions and 
    +    limitations under the License.
    +-->
    +
    +#S3 Guide (Alpha Feature 1.4.1)
    +Amazon S3 is a cloud storage service that is recommended for storing large data files. You can 
    +use this feature if you want to store data on amazon cloud. Since the data is stored on to cloud 
    +storage there are no restrictions on the size of data and the data can be accessed from anywhere at any time.
    +Carbon can support any Object store that conforms to Amazon S3 API. 
    +
    +#Writing to Object Store
    +To store carbondata files on to Object Store location, you need to set `carbon
    +.storelocation` property to Object Store path in CarbonProperties file. For example, carbon
    +.storelocation=s3a://mybucket/carbonstore. By setting this property, all the tables will be created on the specified Object Store path.
    +
    +If your existing store is HDFS, and you want to store specific tables on S3 location, then `location` parameter has to be set during create 
    +table. 
    +For example:
    +
    +```
    +CREATE TABLE IF NOT EXISTS db1.table1(col1 string, col2 int) STORED AS carbondata LOCATION 's3a://mybucket/carbonstore'
    +``` 
    +
    +For more details on create table, Refer [data-management-on-carbondata](https://github.com/apache/carbondata/blob/master/docs/data-management-on-carbondata.md#create-table)
    +
    +#Authentication
    +You need to set authentication properties to store the carbondata files on to S3 location. For 
    +more details on authentication properties, refer 
    +[hadoop authentication document](https://hadoop.apache.org/docs/stable/hadoop-aws/tools/hadoop-aws/index.html#Authentication_properties)
    +
    +Another way of setting the authentication parameters is as follows:
    +
    +```
    +         SparkSession
    +         .builder()
    +         .master(masterURL)
    +         .appName("S3Example")
    +         .config("spark.driver.host", "localhost")
    +         .config("spark.hadoop.fs.s3a.access.key", "1111")
    +         .config("spark.hadoop.fs.s3a.secret.key", "2222")
    +         .config("spark.hadoop.fs.s3a.endpoint", "1.1.1.1")
    +         .getOrCreateCarbonSession()
    +```
    +
    +#Recommendations
    +1. Object stores like S3 does not support file leasing mechanism(supported by HDFS) that is 
    +required to take locks which ensure consistency between concurrent operations therefore, it is 
    +recommended to set the configurable lock path property([carbon.lock.path](https://github.com/apache/carbondata/blob/master/docs/configuration-parameters.md#miscellaneous-configuration))
    + to a HDFS directory.
    +2. As Object stores are eventual consistent meaning that any put request can take some time to reflect when trying to list objects from that bucket therefore concurrent queries are not supported. 
    --- End diff --
    
    done


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/6156/



---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by kunal642 <gi...@git.apache.org>.
Github user kunal642 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207249910
  
    --- Diff: docs/s3-guide.md ---
    @@ -0,0 +1,63 @@
    +<!--
    +    Licensed to the Apache Software Foundation (ASF) under one or more 
    +    contributor license agreements.  See the NOTICE file distributed with
    +    this work for additional information regarding copyright ownership. 
    +    The ASF licenses this file to you under the Apache License, Version 2.0
    +    (the "License"); you may not use this file except in compliance with 
    +    the License.  You may obtain a copy of the License at
    +
    +      http://www.apache.org/licenses/LICENSE-2.0
    +
    +    Unless required by applicable law or agreed to in writing, software 
    +    distributed under the License is distributed on an "AS IS" BASIS, 
    +    WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
    +    See the License for the specific language governing permissions and 
    +    limitations under the License.
    +-->
    +
    +#S3 Guide (Alpha Feature 1.4.1)
    +Amazon S3 is a cloud storage service that is recommended for storing large data files. You can 
    --- End diff --
    
    done


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    LGTM


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/7768/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Failed with Spark 2.2.1, Please check CI http://88.99.58.216:8080/job/ApacheCarbonPRBuilder/6410/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/7773/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    SDV Build Success , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/6123/



---

[GitHub] carbondata pull request #2576: [CARBONDATA-2795] Add documentation for S3

Posted by kunal642 <gi...@git.apache.org>.
Github user kunal642 commented on a diff in the pull request:

    https://github.com/apache/carbondata/pull/2576#discussion_r207249973
  
    --- Diff: docs/data-management-on-carbondata.md ---
    @@ -730,6 +736,8 @@ Users can specify which columns to include and exclude for local dictionary gene
       * If the IGNORE option is used, then bad records are neither loaded nor written to the separate CSV file.
       * In loaded data, if all records are bad records, the BAD_RECORDS_ACTION is invalid and the load operation fails.
       * The maximum number of characters per column is 32000. If there are more than 32000 characters in a column, data loading will fail.
    +  * Since Bad Records Path can be specified in both create, load and carbon properties. 
    --- End diff --
    
    done


---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Failed  with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/7723/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by CarbonDataQA <gi...@git.apache.org>.
Github user CarbonDataQA commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    Build Success with Spark 2.1.0, Please check CI http://136.243.101.176:8080/job/ApacheCarbonPRBuilder1/7776/



---

[GitHub] carbondata issue #2576: [CARBONDATA-2795] Add documentation for S3

Posted by ravipesala <gi...@git.apache.org>.
Github user ravipesala commented on the issue:

    https://github.com/apache/carbondata/pull/2576
  
    SDV Build Fail , Please check CI http://144.76.159.231:8080/job/ApacheSDVTests/6151/



---