[GitHub] carbondata issue #2850: [WIP] Added concurrent reading through SDK
Github user NamanRastogi commented on the issue: https://github.com/apache/carbondata/pull/2850 Please check the split method, it splits the list of `CarbonRecordReader` into multiple `CarbonReader`s. It does not jumble the order of `CarbonRecordReader`, it still keeps them sequential. Suppose there are 10 *carbondata* files and thus 10 `CarbonRecordReader` in `CarbonReader.readers` object and the user wants to get 3 splits, so he will get a list like this: ```java CarbonReader reader = CarbonReader.builder(dataDir).build(); List multipleReaders = reader.split(3); ``` And the indices of `CarbonRecordReader`s in `multipleReaders` will be like: `multipleReaders.get(0).readers` points to {0,1,2,3} indices of *carbondata* files `multipleReaders.get(1).readers` points to {4,5,6} indices of *carbondata* files `multipleReaders.get(2).readers` points to {7,8,9} indices of *carbondata* files Now, if you read the rows like following code, the rows will still be in order. ```java for (CarbonReader reader_i : multipleReaders) { reader_i.readNextRow(); } ``` Earlier, you were getting data from 5th `CarbonRecordReader` only after you have exhausted the 4th. But now, you are getting it earlier, maybe even before 0th. So the user has to make sure he consumes it after he has used up the 4th file if order is important for him/her, otherwise he/she can use it earlier also if order is not important. So, for example to count the total no. of rows, user does not need the original order. ---
[GitHub] carbondata issue #2850: [WIP] Added concurrent reading through SDK
Github user NamanRastogi commented on the issue: https://github.com/apache/carbondata/pull/2850 Yes, data coming from one file will always be in order. Please check the `split` method, it splits the list of CarbonRecordReader into multiple CarbonReader s. Suppose there are 10 carbondata files, and the user wants to get 3 splits, so he will get a list like this: ---
[GitHub] carbondata issue #2850: [WIP] Added concurrent reading through SDK
Github user xubo245 commented on the issue: https://github.com/apache/carbondata/pull/2850 @NamanRastogi Hi, customer required carbon provide the same order between using one thread to read and use multiple threads to read data. ---
[GitHub] carbondata issue #2850: [WIP] Added concurrent reading through SDK
Github user NamanRastogi commented on the issue: https://github.com/apache/carbondata/pull/2850 @xuchuanyin Using this API the overhead of reading concurrently is on the consumer of CarbonReader. So yes, what you said what you said is right. But this API was made to consider this case only when a user wants to have more manual control over the concurrent reading. Consider the scenario when the user wants to read different files in different machines. If we take care of the concurrent reading internally (using a buffer as you suggested), the user cannot do that. Another thing that you mentioned was user providing the location of _carbondata_ files in the SDK reader itself, and that does not work in the current implementation of SDK. And for faster reading, there is one more pull request: [2816](https://github.com/apache/carbondata/pull/2816), it supports batch reading for a better reading performance. It is still not concurrent reading, but better than iterator based row-by-row reading. ---
[GitHub] carbondata issue #2850: [WIP] Added concurrent reading through SDK
Github user xuchuanyin commented on the issue: https://github.com/apache/carbondata/pull/2850 emm, but in your implementation, most of the work has to be done by the user (multi-thread handling). CarbonData itself only split the input data and return multiple readers. If this is the solution, why not just tell the user to generate multiple CarbonReaders by passing only part of the input dir each time they create the reader? Addition to my proposal, I think we can add a buffer for the records. When `CarbonReader.next` is called, we can retrieve the record from the buffer and fill the buffer asynchronously. When`CarbonReader.hasNext` is called, we can first detect this from the buffer, if it is empty, we will then detect this from the recordReader and fill the buffer asynchronously. ---
[GitHub] carbondata issue #2850: [WIP] Added concurrent reading through SDK
Github user NamanRastogi commented on the issue: https://github.com/apache/carbondata/pull/2850 @xuchuanyin Since `CarbonReader` is iterator based, we can only read a line when user wants. So, even if we keep the parallelism internally before the build method, it will still read the file row by row (even though they are in different threads) and reading multiple rows will still be accessed sequentially. Please notice the test file **ConcurrentSdkReaderTest.java**, the reading is happening in the thread itself, and multiple threads are reading the files (line-by-line) concurrently. The actual reading is happening inside the `CarbonReader.readNext()` method, as long as that is sequential, the actual reading performance is not going to get better, so we have to have `CarbonReader.readNext()` inside different threads,. ---
[GitHub] carbondata issue #2850: [WIP] Added concurrent reading through SDK
Github user xuchuanyin commented on the issue: https://github.com/apache/carbondata/pull/2850 @NamanRastogi I think we can further optimize this function. 1. We can enable the parallel reading and set the parallelism while creating a CarbonReader; 2. Inside CarbonReader, we handle the concurrent processing; 3. The interfaces for CarbonReader should be kept the same as before, there is no need to add more interfaces. By calling hasNext or next, user can get the next record and will not care about which RecordReader does this record belong to. The user interface looks like below: ``` CarbonReader reader = CarbonReader.builder(dataDir).parallelism(3).build(); while (reader.hasNext()) { reader.next(); } reader.close(); ``` To keep it simple, by default the parallelism can be 1 which means we will process each RecordReader one by one. Setting this parallelism to a higher value means that we will go process the RecordReaders in a thread pool with size 3. ---
[GitHub] carbondata issue #2850: [WIP] Added concurrent reading through SDK
Github user CarbonDataQA commented on the issue: https://github.com/apache/carbondata/pull/2850 Can one of the admins verify this patch? ---