Skip to content

Commit

Permalink
[CARBONDATA-3246]Fix sdk reader issue if batch size is given as zero …
Browse files Browse the repository at this point in the history
…and vectorRead False

Problem: SDK reader is failing if vectorRead is false and detail query batch
size is given as 0.Compiler is giving stack overflow error after getting stuck
in ChunkRowIterator.hasnext recurssion.
Solution: Since 0 is wrong batch size, we should take DETAIL_QUERY_BATCH_SIZE_DEFAULT
as the batch size.

This closes #3070
  • Loading branch information
shardul-cr7 authored and kunal642 committed Jan 18, 2019
1 parent d51cbc5 commit af297a9
Show file tree
Hide file tree
Showing 2 changed files with 42 additions and 0 deletions.
Expand Up @@ -94,6 +94,9 @@ public abstract class AbstractDetailQueryResultIterator<E> extends CarbonIterato
if (null != batchSizeString) {
try {
batchSize = Integer.parseInt(batchSizeString);
if (0 == batchSize) {
batchSize = CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE_DEFAULT;
}
} catch (NumberFormatException ne) {
LOGGER.error("Invalid inmemory records size. Using default value");
batchSize = CarbonCommonConstants.DETAIL_QUERY_BATCH_SIZE_DEFAULT;
Expand Down
Expand Up @@ -132,6 +132,45 @@ private void writeDataMultipleFiles(int numFiles, long numRowsPerFile) {
}
}

@Test public void testReadWithZeroBatchSize() throws InterruptedException {
int numFiles = 5;
int numRowsPerFile = 5;
short numThreads = 4;
writeDataMultipleFiles(numFiles, numRowsPerFile);

// Concurrent Reading
ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
try {
long count;
CarbonReader reader =
CarbonReader.builder(dataDir).withRowRecordReader().withBatch(0).build();
List<CarbonReader> multipleReaders = reader.split(numThreads);
try {
List<ReadLogic> tasks = new ArrayList<>();
List<Future<Long>> results;
count = 0;

for (CarbonReader reader_i : multipleReaders) {
tasks.add(new ReadLogic(reader_i));
}
results = executorService.invokeAll(tasks);
for (Future result_i : results) {
count += (long) result_i.get();
}
Assert.assertEquals(numFiles * numRowsPerFile, count);
} catch (Exception e) {
e.printStackTrace();
Assert.fail(e.getMessage());
}
} catch (Exception e) {
e.printStackTrace();
Assert.fail(e.getMessage());
} finally {
executorService.shutdown();
executorService.awaitTermination(10, TimeUnit.MINUTES);
}
}

class ReadLogic implements Callable<Long> {
CarbonReader reader;

Expand Down

0 comments on commit af297a9

Please sign in to comment.