You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ku...@apache.org on 2019/01/18 19:04:49 UTC

[carbondata] branch master updated: [CARBONDATA-3246]Fix sdk reader issue if batch size is given as zero and vectorRead False

This is an automated email from the ASF dual-hosted git repository.

kunalkapoor pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/carbondata.git


The following commit(s) were added to refs/heads/master by this push:
     new af297a9  [CARBONDATA-3246]Fix sdk reader issue if batch size is given as zero and vectorRead False
af297a9 is described below

commit af297a9b80857265e2473273daeff971f34db3ee
Author: shardul-cr7 <sh...@gmail.com>
AuthorDate: Fri Jan 11 16:10:27 2019 +0530

    [CARBONDATA-3246]Fix sdk reader issue if batch size is given as zero 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
---
 .../AbstractDetailQueryResultIterator.java         |  3 ++
 .../sdk/file/ConcurrentSdkReaderTest.java          | 39 ++++++++++++++++++++++
 2 files changed, 42 insertions(+)

diff --git a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
index 9282d44..30f5183 100644
--- a/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
+++ b/core/src/main/java/org/apache/carbondata/core/scan/result/iterator/AbstractDetailQueryResultIterator.java
@@ -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;
diff --git a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/ConcurrentSdkReaderTest.java b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/ConcurrentSdkReaderTest.java
index fef3319..31342b9 100644
--- a/store/sdk/src/test/java/org/apache/carbondata/sdk/file/ConcurrentSdkReaderTest.java
+++ b/store/sdk/src/test/java/org/apache/carbondata/sdk/file/ConcurrentSdkReaderTest.java
@@ -132,6 +132,45 @@ public class ConcurrentSdkReaderTest {
     }
   }
 
+  @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;