You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@carbondata.apache.org by GitBox <gi...@apache.org> on 2020/02/17 13:11:15 UTC

[GitHub] [carbondata] jackylk opened a new pull request #3627: [CARBONDATA-3710] Make stage files queryable

jackylk opened a new pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627
 
 
    ### Why is this PR needed?
   Currenlty, stage files are written by SDK to a temp folder inside carbondata table path, but it is only queryable after calling "INSERT INTO tableName STAGE" command, it makes data latency longer.
    
    ### What changes were proposed in this PR?
   This PR is to add a feature to make stage files queryable before inserting them into the table.
   Since these stage files do not have index, they need to be scanned like normal columnar files
   
   
    ### Does this PR introduce any user interface change?
    - No
   
   
    ### Is any new testcase added?
    - No (current testcase is enhanced to test querying the stage files, see TestCarbonWriter)
   
       
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383675440
 
 

 ##########
 File path: docs/configuration-parameters.md
 ##########
 @@ -144,6 +144,7 @@ This section provides the details of all the configurations required for the Car
 | carbon.heap.memory.pooling.threshold.bytes | 1048576 | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. Using unsafe, memory can be allocated on Java Heap or off heap. This configuration controls the allocation mechanism on Java HEAP. If the heap memory allocations of the given size is greater or equal than this value,it should go through the pooling mechanism. But if set this size to -1, it should not go through the pooling mechanism. Default value is 1048576(1MB, the same as Spark). Value to be specified in bytes. |
 | carbon.push.rowfilters.for.vector | false | When enabled complete row filters will be handled by carbon in case of vector. If it is disabled then only page level pruning will be done by carbon and row level filtering will be done by spark for vector. And also there are scan optimizations in carbon to avoid multiple data copies when this parameter is set to false. There is no change in flow for non-vector based queries. |
 | carbon.query.prefetch.enable | true | By default this property is true, so prefetch is used in query to read next blocklet asynchronously in other thread while processing current blocklet in main thread. This can help to reduce CPU idle time. Setting this property false will disable this prefetch feature in query. |
+| carbon.query.stage.input.enable | false | Stage input files are data files written by external applications (such as Flink) but has not been loaded into carbon table. Enabling this configuration makes query includes these files, thus makes query on latest data. However, since these files are not indexed, query maybe slower due to full scan is required for these files. |
 
 Review comment:
   @niuge01 and @jackylk : can we please have a flink guide explains the integration of carbondata with flink. 
   Also we can add about stage files folder structure, file name , location and format in the same document.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#issuecomment-587029007
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2023/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383726755
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/statusmanager/StageInputCollector.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.
+ */
+
+package org.apache.carbondata.core.statusmanager;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+
+import static org.apache.carbondata.core.util.path.CarbonTablePath.SUCCESS_FILE_SUBFIX;
+
+import com.google.gson.Gson;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.log4j.Logger;
+
+/**
+ * Utilities to create input split from stage files
+ */
+public class StageInputCollector {
+
+  private static Logger LOGGER =
+      LogServiceFactory.getLogService(StageInputCollector.class.getCanonicalName());
+
+  /**
+   * Collect all stage files and create splits from them.
+   * These splits can be included in the queried.
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#issuecomment-586995193
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/321/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383714128
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -1519,6 +1519,12 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_QUERY_PREFETCH_ENABLE_DEFAULT = "true";
 
+  @CarbonProperty
+  public static final String CARBON_QUERY_STAGE_INPUT =
+      "carbon.query.stage.input.enable";
+
+  public static final String CARBON_QUERY_STAGE_INPUT_DEFAULT = "false";
 
 Review comment:
   I'd like to make it disable by default because it may impact the query performance. Stage files are not sorted and need to be scanned

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383725524
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/statusmanager/StageInputCollector.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.
+ */
+
+package org.apache.carbondata.core.statusmanager;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+
+import static org.apache.carbondata.core.util.path.CarbonTablePath.SUCCESS_FILE_SUBFIX;
+
+import com.google.gson.Gson;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.log4j.Logger;
+
+/**
+ * Utilities to create input split from stage files
+ */
+public class StageInputCollector {
+
+  private static Logger LOGGER =
+      LogServiceFactory.getLogService(StageInputCollector.class.getCanonicalName());
+
+  /**
+   * Collect all stage files and create splits from them.
+   * These splits can be included in the queried.
+   */
+  public static List<InputSplit> createInputSplits(CarbonTable table, Configuration hadoopConf)
+      throws ExecutionException, InterruptedException {
+    List<CarbonFile> stageInputFiles = new LinkedList<>();
+    List<CarbonFile> successFiles = new LinkedList<>();
+    collectStageFiles(table, hadoopConf, stageInputFiles, successFiles);
+    if (stageInputFiles.size() > 0) {
+      int numThreads = Math.min(Math.max(stageInputFiles.size(), 1), 10);
+      ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
+      return createInputSplits(executorService, stageInputFiles);
+    } else {
+      return new ArrayList<>(0);
+    }
+  }
+
+  /**
+   * Collect all stage files and matched success files.
+   * A stage file without success file will not be collected
+   */
+  public static void collectStageFiles(CarbonTable table, Configuration hadoopConf,
+      List<CarbonFile> stageInputList, List<CarbonFile> successFileList) {
+    Objects.requireNonNull(table);
+    Objects.requireNonNull(hadoopConf);
+    Objects.requireNonNull(stageInputList);
+    Objects.requireNonNull(successFileList);
+    CarbonFile dir = FileFactory.getCarbonFile(table.getStagePath(), hadoopConf);
+    if (dir.exists()) {
+      // list the stage folder and collect all stage files who has corresponding success file,
+      // which means the file is committed
+      CarbonFile[] allFiles = dir.listFiles();
 
 Review comment:
   But this way can not access S3, right?
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on issue #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on issue #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#issuecomment-590849586
 
 
   LGTM

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#issuecomment-590464008
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2139/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383680405
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/statusmanager/StageInputCollector.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.
+ */
+
+package org.apache.carbondata.core.statusmanager;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+
+import static org.apache.carbondata.core.util.path.CarbonTablePath.SUCCESS_FILE_SUBFIX;
+
+import com.google.gson.Gson;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.log4j.Logger;
+
+/**
+ * Utilities to create input split from stage files
+ */
+public class StageInputCollector {
+
+  private static Logger LOGGER =
+      LogServiceFactory.getLogService(StageInputCollector.class.getCanonicalName());
+
+  /**
+   * Collect all stage files and create splits from them.
+   * These splits can be included in the queried.
+   */
+  public static List<InputSplit> createInputSplits(CarbonTable table, Configuration hadoopConf)
+      throws ExecutionException, InterruptedException {
+    List<CarbonFile> stageInputFiles = new LinkedList<>();
+    List<CarbonFile> successFiles = new LinkedList<>();
+    collectStageFiles(table, hadoopConf, stageInputFiles, successFiles);
+    if (stageInputFiles.size() > 0) {
+      int numThreads = Math.min(Math.max(stageInputFiles.size(), 1), 10);
+      ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
+      return createInputSplits(executorService, stageInputFiles);
+    } else {
+      return new ArrayList<>(0);
+    }
+  }
+
+  /**
+   * Collect all stage files and matched success files.
+   * A stage file without success file will not be collected
+   */
+  public static void collectStageFiles(CarbonTable table, Configuration hadoopConf,
+      List<CarbonFile> stageInputList, List<CarbonFile> successFileList) {
+    Objects.requireNonNull(table);
+    Objects.requireNonNull(hadoopConf);
+    Objects.requireNonNull(stageInputList);
+    Objects.requireNonNull(successFileList);
+    CarbonFile dir = FileFactory.getCarbonFile(table.getStagePath(), hadoopConf);
+    if (dir.exists()) {
+      // list the stage folder and collect all stage files who has corresponding success file,
+      // which means the file is committed
+      CarbonFile[] allFiles = dir.listFiles();
 
 Review comment:
   i think, better to avoid listfiles to avoid any future problems. it will create many file objects once, we can use nio APIs. like below
   `Files.newDirectoryStream(Paths.get(table.getStagePath()), "*.success").forEach(path -> {
           map.put(key, value);
         }
         );`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383670008
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -1519,6 +1519,12 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_QUERY_PREFETCH_ENABLE_DEFAULT = "true";
 
+  @CarbonProperty
+  public static final String CARBON_QUERY_STAGE_INPUT =
+      "carbon.query.stage.input.enable";
+
+  public static final String CARBON_QUERY_STAGE_INPUT_DEFAULT = "false";
 
 Review comment:
   default should be `true` ? because it can help avoid latency by default

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#issuecomment-590418301
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/439/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383669888
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -1519,6 +1519,12 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_QUERY_PREFETCH_ENABLE_DEFAULT = "true";
 
+  @CarbonProperty
+  public static final String CARBON_QUERY_STAGE_INPUT =
 
 Review comment:
   need support dynamicConfigurable ? else user need to restart the cluster

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] asfgit closed pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627
 
 
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383674740
 
 

 ##########
 File path: docs/configuration-parameters.md
 ##########
 @@ -144,6 +144,7 @@ This section provides the details of all the configurations required for the Car
 | carbon.heap.memory.pooling.threshold.bytes | 1048576 | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. Using unsafe, memory can be allocated on Java Heap or off heap. This configuration controls the allocation mechanism on Java HEAP. If the heap memory allocations of the given size is greater or equal than this value,it should go through the pooling mechanism. But if set this size to -1, it should not go through the pooling mechanism. Default value is 1048576(1MB, the same as Spark). Value to be specified in bytes. |
 | carbon.push.rowfilters.for.vector | false | When enabled complete row filters will be handled by carbon in case of vector. If it is disabled then only page level pruning will be done by carbon and row level filtering will be done by spark for vector. And also there are scan optimizations in carbon to avoid multiple data copies when this parameter is set to false. There is no change in flow for non-vector based queries. |
 | carbon.query.prefetch.enable | true | By default this property is true, so prefetch is used in query to read next blocklet asynchronously in other thread while processing current blocklet in main thread. This can help to reduce CPU idle time. Setting this property false will disable this prefetch feature in query. |
+| carbon.query.stage.input.enable | false | Stage input files are data files written by external applications (such as Flink) but has not been loaded into carbon table. Enabling this configuration makes query includes these files, thus makes query on latest data. However, since these files are not indexed, query maybe slower due to full scan is required for these files. |
 
 Review comment:
   Pushed down filter cases , how it is handled ? for non-stage files carbon applies the filtering, but stage files cannot be filtered. spark may not apply filter for carbon results because filter is already pushed down to carbon ?
   
   block pruning identifies all the stage files as splits and filter applied while blocklet pruning ? 

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383716671
 
 

 ##########
 File path: integration/flink/src/test/scala/org/apache/carbon/flink/TestCarbonWriter.scala
 ##########
 @@ -95,9 +96,18 @@ class TestCarbonWriter extends QueryTest {
           throw new UnsupportedOperationException(exception)
       }
 
+      checkAnswer(sql(s"select count(1) from $tableName"), Seq(Row(0)))
+
+      // query with stage input
+      CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_QUERY_STAGE_INPUT, "true")
 
 Review comment:
   added

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] niuge01 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
niuge01 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#issuecomment-587235685
 
 
   LGTM

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383674740
 
 

 ##########
 File path: docs/configuration-parameters.md
 ##########
 @@ -144,6 +144,7 @@ This section provides the details of all the configurations required for the Car
 | carbon.heap.memory.pooling.threshold.bytes | 1048576 | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. Using unsafe, memory can be allocated on Java Heap or off heap. This configuration controls the allocation mechanism on Java HEAP. If the heap memory allocations of the given size is greater or equal than this value,it should go through the pooling mechanism. But if set this size to -1, it should not go through the pooling mechanism. Default value is 1048576(1MB, the same as Spark). Value to be specified in bytes. |
 | carbon.push.rowfilters.for.vector | false | When enabled complete row filters will be handled by carbon in case of vector. If it is disabled then only page level pruning will be done by carbon and row level filtering will be done by spark for vector. And also there are scan optimizations in carbon to avoid multiple data copies when this parameter is set to false. There is no change in flow for non-vector based queries. |
 | carbon.query.prefetch.enable | true | By default this property is true, so prefetch is used in query to read next blocklet asynchronously in other thread while processing current blocklet in main thread. This can help to reduce CPU idle time. Setting this property false will disable this prefetch feature in query. |
+| carbon.query.stage.input.enable | false | Stage input files are data files written by external applications (such as Flink) but has not been loaded into carbon table. Enabling this configuration makes query includes these files, thus makes query on latest data. However, since these files are not indexed, query maybe slower due to full scan is required for these files. |
 
 Review comment:
   Pushed down filter cases , how it is handled ? for non-stage files carbon applies the filtering, but stage files cannot be filtered. spark may not apply filter for carbon results because filter is already pushed down to carbon ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383680782
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/statusmanager/StageInputCollector.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.
+ */
+
+package org.apache.carbondata.core.statusmanager;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+
+import static org.apache.carbondata.core.util.path.CarbonTablePath.SUCCESS_FILE_SUBFIX;
+
+import com.google.gson.Gson;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.log4j.Logger;
+
+/**
+ * Utilities to create input split from stage files
+ */
+public class StageInputCollector {
+
+  private static Logger LOGGER =
+      LogServiceFactory.getLogService(StageInputCollector.class.getCanonicalName());
+
+  /**
+   * Collect all stage files and create splits from them.
+   * These splits can be included in the queried.
+   */
+  public static List<InputSplit> createInputSplits(CarbonTable table, Configuration hadoopConf)
+      throws ExecutionException, InterruptedException {
+    List<CarbonFile> stageInputFiles = new LinkedList<>();
+    List<CarbonFile> successFiles = new LinkedList<>();
+    collectStageFiles(table, hadoopConf, stageInputFiles, successFiles);
+    if (stageInputFiles.size() > 0) {
+      int numThreads = Math.min(Math.max(stageInputFiles.size(), 1), 10);
+      ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
+      return createInputSplits(executorService, stageInputFiles);
+    } else {
+      return new ArrayList<>(0);
 
 Review comment:
   Just give ` new ArrayList<>()`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383712889
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -1519,6 +1519,12 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_QUERY_PREFETCH_ENABLE_DEFAULT = "true";
 
+  @CarbonProperty
+  public static final String CARBON_QUERY_STAGE_INPUT =
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383717477
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ##########
 @@ -501,6 +501,13 @@ public String getMetadataPath() {
     return CarbonTablePath.getMetadataPath(getTablePath());
   }
 
+  /**
+   * Return the stage input path
+   */
+  public String getStagePath() {
 
 Review comment:
   CarbonTablePath.`SNAPSHOT_FILE_NAME` is not used. can we remove it ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383677243
 
 

 ##########
 File path: integration/flink/src/test/scala/org/apache/carbon/flink/TestCarbonWriter.scala
 ##########
 @@ -95,9 +96,18 @@ class TestCarbonWriter extends QueryTest {
           throw new UnsupportedOperationException(exception)
       }
 
+      checkAnswer(sql(s"select count(1) from $tableName"), Seq(Row(0)))
+
+      // query with stage input
+      CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_QUERY_STAGE_INPUT, "true")
 
 Review comment:
   can we add one filter query test case also ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383715545
 
 

 ##########
 File path: docs/configuration-parameters.md
 ##########
 @@ -144,6 +144,7 @@ This section provides the details of all the configurations required for the Car
 | carbon.heap.memory.pooling.threshold.bytes | 1048576 | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. Using unsafe, memory can be allocated on Java Heap or off heap. This configuration controls the allocation mechanism on Java HEAP. If the heap memory allocations of the given size is greater or equal than this value,it should go through the pooling mechanism. But if set this size to -1, it should not go through the pooling mechanism. Default value is 1048576(1MB, the same as Spark). Value to be specified in bytes. |
 | carbon.push.rowfilters.for.vector | false | When enabled complete row filters will be handled by carbon in case of vector. If it is disabled then only page level pruning will be done by carbon and row level filtering will be done by spark for vector. And also there are scan optimizations in carbon to avoid multiple data copies when this parameter is set to false. There is no change in flow for non-vector based queries. |
 | carbon.query.prefetch.enable | true | By default this property is true, so prefetch is used in query to read next blocklet asynchronously in other thread while processing current blocklet in main thread. This can help to reduce CPU idle time. Setting this property false will disable this prefetch feature in query. |
+| carbon.query.stage.input.enable | false | Stage input files are data files written by external applications (such as Flink) but has not been loaded into carbon table. Enabling this configuration makes query includes these files, thus makes query on latest data. However, since these files are not indexed, query maybe slower due to full scan is required for these files. |
 
 Review comment:
   In driver getSplits, carbon will return all stage files for scanning. But when scanning the file in executor side, carbon reader will get the filter from spark, and it applies the filter inside the reader when reading the stage files

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383717477
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ##########
 @@ -501,6 +501,13 @@ public String getMetadataPath() {
     return CarbonTablePath.getMetadataPath(getTablePath());
   }
 
+  /**
+   * Return the stage input path
+   */
+  public String getStagePath() {
 
 Review comment:
   In CarbonTablePath class, `SNAPSHOT_FILE_NAME` is not used. can we remove it ?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383728166
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ##########
 @@ -501,6 +501,13 @@ public String getMetadataPath() {
     return CarbonTablePath.getMetadataPath(getTablePath());
   }
 
+  /**
+   * Return the stage input path
+   */
+  public String getStagePath() {
 
 Review comment:
   It is used in func `getStageSnapshotFile`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383730022
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
 ##########
 @@ -501,6 +501,13 @@ public String getMetadataPath() {
     return CarbonTablePath.getMetadataPath(getTablePath());
   }
 
+  /**
+   * Return the stage input path
+   */
+  public String getStagePath() {
 
 Review comment:
   I know, that function is used only for test files. I didn't see `snapshot` file creation in the flink sdk write code.

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383727646
 
 

 ##########
 File path: docs/configuration-parameters.md
 ##########
 @@ -144,6 +144,7 @@ This section provides the details of all the configurations required for the Car
 | carbon.heap.memory.pooling.threshold.bytes | 1048576 | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. Using unsafe, memory can be allocated on Java Heap or off heap. This configuration controls the allocation mechanism on Java HEAP. If the heap memory allocations of the given size is greater or equal than this value,it should go through the pooling mechanism. But if set this size to -1, it should not go through the pooling mechanism. Default value is 1048576(1MB, the same as Spark). Value to be specified in bytes. |
 | carbon.push.rowfilters.for.vector | false | When enabled complete row filters will be handled by carbon in case of vector. If it is disabled then only page level pruning will be done by carbon and row level filtering will be done by spark for vector. And also there are scan optimizations in carbon to avoid multiple data copies when this parameter is set to false. There is no change in flow for non-vector based queries. |
 | carbon.query.prefetch.enable | true | By default this property is true, so prefetch is used in query to read next blocklet asynchronously in other thread while processing current blocklet in main thread. This can help to reduce CPU idle time. Setting this property false will disable this prefetch feature in query. |
+| carbon.query.stage.input.enable | false | Stage input files are data files written by external applications (such as Flink) but has not been loaded into carbon table. Enabling this configuration makes query includes these files, thus makes query on latest data. However, since these files are not indexed, query maybe slower due to full scan is required for these files. |
 
 Review comment:
   fixed

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#issuecomment-590757484
 
 
   Build Success with Spark 2.4.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/458/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383669888
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -1519,6 +1519,12 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_QUERY_PREFETCH_ENABLE_DEFAULT = "true";
 
+  @CarbonProperty
+  public static final String CARBON_QUERY_STAGE_INPUT =
 
 Review comment:
   need support dynamicConfigurable ? It is used in session params

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] CarbonDataQA1 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
CarbonDataQA1 commented on issue #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#issuecomment-590795061
 
 
   Build Success with Spark 2.3.4, Please check CI http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2159/
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
ajantha-bhat commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383669888
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##########
 @@ -1519,6 +1519,12 @@ private CarbonCommonConstants() {
 
   public static final String CARBON_QUERY_PREFETCH_ENABLE_DEFAULT = "true";
 
+  @CarbonProperty
+  public static final String CARBON_QUERY_STAGE_INPUT =
 
 Review comment:
   please annotate as `dynamicConfigurable` ? It is used in session params

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
jackylk commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383726240
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/statusmanager/StageInputCollector.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.
+ */
+
+package org.apache.carbondata.core.statusmanager;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+
+import static org.apache.carbondata.core.util.path.CarbonTablePath.SUCCESS_FILE_SUBFIX;
+
+import com.google.gson.Gson;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.log4j.Logger;
+
+/**
+ * Utilities to create input split from stage files
+ */
+public class StageInputCollector {
+
+  private static Logger LOGGER =
+      LogServiceFactory.getLogService(StageInputCollector.class.getCanonicalName());
+
+  /**
+   * Collect all stage files and create splits from them.
+   * These splits can be included in the queried.
+   */
+  public static List<InputSplit> createInputSplits(CarbonTable table, Configuration hadoopConf)
+      throws ExecutionException, InterruptedException {
+    List<CarbonFile> stageInputFiles = new LinkedList<>();
+    List<CarbonFile> successFiles = new LinkedList<>();
+    collectStageFiles(table, hadoopConf, stageInputFiles, successFiles);
+    if (stageInputFiles.size() > 0) {
+      int numThreads = Math.min(Math.max(stageInputFiles.size(), 1), 10);
+      ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
+      return createInputSplits(executorService, stageInputFiles);
+    } else {
+      return new ArrayList<>(0);
 
 Review comment:
   It is ok, avoiding creating more objects

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383683722
 
 

 ##########
 File path: docs/configuration-parameters.md
 ##########
 @@ -144,6 +144,7 @@ This section provides the details of all the configurations required for the Car
 | carbon.heap.memory.pooling.threshold.bytes | 1048576 | CarbonData supports unsafe operations of Java to avoid GC overhead for certain operations. Using unsafe, memory can be allocated on Java Heap or off heap. This configuration controls the allocation mechanism on Java HEAP. If the heap memory allocations of the given size is greater or equal than this value,it should go through the pooling mechanism. But if set this size to -1, it should not go through the pooling mechanism. Default value is 1048576(1MB, the same as Spark). Value to be specified in bytes. |
 | carbon.push.rowfilters.for.vector | false | When enabled complete row filters will be handled by carbon in case of vector. If it is disabled then only page level pruning will be done by carbon and row level filtering will be done by spark for vector. And also there are scan optimizations in carbon to avoid multiple data copies when this parameter is set to false. There is no change in flow for non-vector based queries. |
 | carbon.query.prefetch.enable | true | By default this property is true, so prefetch is used in query to read next blocklet asynchronously in other thread while processing current blocklet in main thread. This can help to reduce CPU idle time. Setting this property false will disable this prefetch feature in query. |
+| carbon.query.stage.input.enable | false | Stage input files are data files written by external applications (such as Flink) but has not been loaded into carbon table. Enabling this configuration makes query includes these files, thus makes query on latest data. However, since these files are not indexed, query maybe slower due to full scan is required for these files. |
 
 Review comment:
   1.add comma `Flink),` and change `has not been` to `have not been`
   
   2.change `query includes these files` to`query to include these files`
   3. change `query maybe slower due to full scan is` to `query maybe slower as full scan is`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383682460
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/statusmanager/StageInputCollector.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.
+ */
+
+package org.apache.carbondata.core.statusmanager;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+
+import static org.apache.carbondata.core.util.path.CarbonTablePath.SUCCESS_FILE_SUBFIX;
+
+import com.google.gson.Gson;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.log4j.Logger;
+
+/**
+ * Utilities to create input split from stage files
+ */
+public class StageInputCollector {
+
+  private static Logger LOGGER =
+      LogServiceFactory.getLogService(StageInputCollector.class.getCanonicalName());
+
+  /**
+   * Collect all stage files and create splits from them.
+   * These splits can be included in the queried.
 
 Review comment:
   change to `These splits can be queries.` or `These splits can be included for the query.`

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

[GitHub] [carbondata] akashrn5 commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #3627: [CARBONDATA-3710] Make stage files queryable
URL: https://github.com/apache/carbondata/pull/3627#discussion_r383729909
 
 

 ##########
 File path: core/src/main/java/org/apache/carbondata/core/statusmanager/StageInputCollector.java
 ##########
 @@ -0,0 +1,143 @@
+/*
+ * 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.
+ */
+
+package org.apache.carbondata.core.statusmanager;
+
+import java.io.IOException;
+import java.io.InputStreamReader;
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Collections;
+import java.util.HashMap;
+import java.util.LinkedList;
+import java.util.List;
+import java.util.Map;
+import java.util.Objects;
+import java.util.concurrent.ExecutionException;
+import java.util.concurrent.ExecutorService;
+import java.util.concurrent.Executors;
+import java.util.concurrent.Future;
+import java.util.stream.Collectors;
+
+import org.apache.carbondata.common.logging.LogServiceFactory;
+import org.apache.carbondata.core.datastore.filesystem.CarbonFile;
+import org.apache.carbondata.core.datastore.impl.FileFactory;
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
+
+import static org.apache.carbondata.core.util.path.CarbonTablePath.SUCCESS_FILE_SUBFIX;
+
+import com.google.gson.Gson;
+import org.apache.commons.io.IOUtils;
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.mapreduce.InputSplit;
+import org.apache.log4j.Logger;
+
+/**
+ * Utilities to create input split from stage files
+ */
+public class StageInputCollector {
+
+  private static Logger LOGGER =
+      LogServiceFactory.getLogService(StageInputCollector.class.getCanonicalName());
+
+  /**
+   * Collect all stage files and create splits from them.
+   * These splits can be included in the queried.
+   */
+  public static List<InputSplit> createInputSplits(CarbonTable table, Configuration hadoopConf)
+      throws ExecutionException, InterruptedException {
+    List<CarbonFile> stageInputFiles = new LinkedList<>();
+    List<CarbonFile> successFiles = new LinkedList<>();
+    collectStageFiles(table, hadoopConf, stageInputFiles, successFiles);
+    if (stageInputFiles.size() > 0) {
+      int numThreads = Math.min(Math.max(stageInputFiles.size(), 1), 10);
+      ExecutorService executorService = Executors.newFixedThreadPool(numThreads);
+      return createInputSplits(executorService, stageInputFiles);
+    } else {
+      return new ArrayList<>(0);
+    }
+  }
+
+  /**
+   * Collect all stage files and matched success files.
+   * A stage file without success file will not be collected
+   */
+  public static void collectStageFiles(CarbonTable table, Configuration hadoopConf,
+      List<CarbonFile> stageInputList, List<CarbonFile> successFileList) {
+    Objects.requireNonNull(table);
+    Objects.requireNonNull(hadoopConf);
+    Objects.requireNonNull(stageInputList);
+    Objects.requireNonNull(successFileList);
+    CarbonFile dir = FileFactory.getCarbonFile(table.getStagePath(), hadoopConf);
+    if (dir.exists()) {
+      // list the stage folder and collect all stage files who has corresponding success file,
+      // which means the file is committed
+      CarbonFile[] allFiles = dir.listFiles();
 
 Review comment:
   can we implement this in our carbonfile API over hadoop conf and try? is it possible?

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services