[CARBONDATA-3710] Make stage files queryable

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?
Yes (One configuration called "carbon.query.stage.input.enable" is added, document added)

Is any new testcase added?
No (current testcase is enhanced to test querying the stage files, see TestCarbonWriter)

This closes #3627
diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index d8194a3..7565555 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -1519,6 +1519,12 @@
 
   public static final String CARBON_QUERY_PREFETCH_ENABLE_DEFAULT = "true";
 
+  @CarbonProperty(dynamicConfigurable = true)
+  public static final String CARBON_QUERY_STAGE_INPUT =
+      "carbon.query.stage.input.enable";
+
+  public static final String CARBON_QUERY_STAGE_INPUT_DEFAULT = "false";
+
   //////////////////////////////////////////////////////////////////////////////////////////
   // Datamap parameter start here
   //////////////////////////////////////////////////////////////////////////////////////////
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
index ccbaf5a..68bdd47 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/CarbonTable.java
@@ -502,6 +502,13 @@
   }
 
   /**
+   * Return the stage input path
+   */
+  public String getStagePath() {
+    return CarbonTablePath.getStageDir(getTablePath());
+  }
+
+  /**
    * Return the segment path of the specified segmentId
    */
   public String getSegmentPath(String segmentId) {
diff --git a/core/src/main/java/org/apache/carbondata/core/statusmanager/StageInputCollector.java b/core/src/main/java/org/apache/carbondata/core/statusmanager/StageInputCollector.java
new file mode 100644
index 0000000..f8e590c
--- /dev/null
+++ b/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 will be included for the query
+   */
+  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();
+      Map<String, CarbonFile> map = new HashMap<>();
+      Arrays.stream(allFiles)
+          .filter(file -> file.getName().endsWith(SUCCESS_FILE_SUBFIX))
+          .forEach(file -> map.put(file.getName().substring(0, file.getName().indexOf(".")), file));
+      Arrays.stream(allFiles)
+          .filter(file -> !file.getName().endsWith(SUCCESS_FILE_SUBFIX))
+          .filter(file -> map.containsKey(file.getName()))
+          .forEach(carbonFile -> {
+            stageInputList.add(carbonFile);
+            successFileList.add(map.get(carbonFile.getName()));
+          });
+    }
+  }
+
+  /**
+   * Read stage files and create input splits from them
+   */
+  public static List<InputSplit> createInputSplits(
+      ExecutorService executorService,
+      List<CarbonFile> stageFiles)
+      throws ExecutionException, InterruptedException {
+    Objects.requireNonNull(executorService);
+    Objects.requireNonNull(stageFiles);
+    long startTime = System.currentTimeMillis();
+    List<InputSplit> output = Collections.synchronizedList(new ArrayList<>());
+    Gson gson = new Gson();
+
+    // read each stage file and create input split
+    // read them using a thread pool to increase parallelism
+    List<Future<Boolean>> futures = stageFiles.stream()
+        .map(stageFile ->
+            executorService.submit(() -> {
+              String filePath = stageFile.getAbsolutePath();
+              InputStreamReader reader = null;
+              try {
+                reader = new InputStreamReader(FileFactory.getDataInputStream(filePath));
+                StageInput stageInput = gson.fromJson(reader, StageInput.class);
+                output.addAll(stageInput.createSplits());
+                return true;
+              } catch (IOException e) {
+                LOGGER.error("failed to read stage file " + filePath);
+                return false;
+              } finally {
+                IOUtils.closeQuietly(reader);
+              }
+            }))
+        .collect(Collectors.toList());
+    for (Future<Boolean> future : futures) {
+      future.get();
+    }
+    LOGGER.info("read stage files taken " + (System.currentTimeMillis() - startTime) + "ms");
+    return output;
+  }
+}
diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
index c6908bb..f322592 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonProperties.java
@@ -1966,4 +1966,10 @@
   public String getDefaultCompressor() {
     return getProperty(CarbonCommonConstants.COMPRESSOR, CarbonCommonConstants.DEFAULT_COMPRESSOR);
   }
+
+  public static boolean isQueryStageInputEnabled() {
+    return Boolean.parseBoolean(getInstance().getProperty(
+        CarbonCommonConstants.CARBON_QUERY_STAGE_INPUT,
+        CarbonCommonConstants.CARBON_QUERY_STAGE_INPUT_DEFAULT));
+  }
 }
diff --git a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
index a193cbe..c9869dd 100644
--- a/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
+++ b/core/src/main/java/org/apache/carbondata/core/util/SessionParams.java
@@ -33,6 +33,7 @@
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_ENABLE_INDEX_SERVER;
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_MAJOR_COMPACTION_SIZE;
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_PUSH_ROW_FILTERS_FOR_VECTOR;
+import static org.apache.carbondata.core.constants.CarbonCommonConstants.CARBON_QUERY_STAGE_INPUT;
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.COMPACTION_SEGMENT_LEVEL_THRESHOLD;
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.ENABLE_AUTO_LOAD_MERGE;
 import static org.apache.carbondata.core.constants.CarbonCommonConstants.ENABLE_OFFHEAP_SORT;
@@ -152,6 +153,7 @@
       case ENABLE_AUTO_LOAD_MERGE:
       case CARBON_PUSH_ROW_FILTERS_FOR_VECTOR:
       case CARBON_ENABLE_INDEX_SERVER:
+      case CARBON_QUERY_STAGE_INPUT:
         isValid = CarbonUtil.validateBoolean(value);
         if (!isValid) {
           throw new InvalidConfigurationException("Invalid value " + value + " for key " + key);
diff --git a/docs/configuration-parameters.md b/docs/configuration-parameters.md
index c90a031..932f3c6 100644
--- a/docs/configuration-parameters.md
+++ b/docs/configuration-parameters.md
@@ -144,6 +144,7 @@
 | 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 have not been loaded into carbon table. Enabling this configuration makes query to include these files, thus makes query on latest data. However, since these files are not indexed, query maybe slower as full scan is required for these files. |
 
 ## Data Mutation Configuration
 | Parameter | Default Value | Description |
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 7381ab0..52dc15c 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -25,6 +25,7 @@
 import java.util.LinkedList;
 import java.util.List;
 import java.util.Map;
+import java.util.concurrent.ExecutionException;
 
 import org.apache.carbondata.common.exceptions.DeprecatedFeatureException;
 import org.apache.carbondata.common.logging.LogServiceFactory;
@@ -55,6 +56,7 @@
 import org.apache.carbondata.core.statusmanager.LoadMetadataDetails;
 import org.apache.carbondata.core.statusmanager.SegmentStatusManager;
 import org.apache.carbondata.core.statusmanager.SegmentUpdateStatusManager;
+import org.apache.carbondata.core.statusmanager.StageInputCollector;
 import org.apache.carbondata.core.stream.StreamFile;
 import org.apache.carbondata.core.stream.StreamPruner;
 import org.apache.carbondata.core.util.CarbonProperties;
@@ -116,6 +118,21 @@
         CarbonCommonConstants.DICTIONARY_INCLUDE)) {
       DeprecatedFeatureException.globalDictNotSupported();
     }
+
+    List<InputSplit> splits = new LinkedList<>();
+
+    if (CarbonProperties.isQueryStageInputEnabled()) {
+      // If there are stage files, collect them and create splits so that they are
+      // included for the query
+      try {
+        List<InputSplit> stageInputSplits =
+            StageInputCollector.createInputSplits(carbonTable, job.getConfiguration());
+        splits.addAll(stageInputSplits);
+      } catch (ExecutionException | InterruptedException e) {
+        LOG.error("Failed to create input splits from stage files", e);
+        throw new IOException(e);
+      }
+    }
     this.readCommittedScope = getReadCommitted(job, identifier);
     LoadMetadataDetails[] loadMetadataDetails = readCommittedScope.getSegmentList();
     String updateDeltaVersion = job.getConfiguration().get(UPDATE_DELTA_VERSION);
@@ -141,12 +158,14 @@
       streamSegments = segments.getStreamSegments();
       streamSegments = getFilteredSegment(job, streamSegments, true, readCommittedScope);
       if (validSegments.size() == 0) {
-        return getSplitsOfStreaming(job, streamSegments, carbonTable);
+        splits.addAll(getSplitsOfStreaming(job, streamSegments, carbonTable));
+        return splits;
       }
       List<Segment> filteredSegmentToAccess =
           getFilteredSegment(job, segments.getValidSegments(), true, readCommittedScope);
       if (filteredSegmentToAccess.size() == 0) {
-        return getSplitsOfStreaming(job, streamSegments, carbonTable);
+        splits.addAll(getSplitsOfStreaming(job, streamSegments, carbonTable));
+        return splits;
       } else {
         setSegmentsToAccess(job.getConfiguration(), filteredSegmentToAccess);
       }
@@ -178,9 +197,10 @@
     }
 
     // do block filtering and get split
-    List<InputSplit> splits = getSplits(
+    List<InputSplit> batchSplits = getSplits(
         job, dataMapFilter, segmentToAccess,
         updateStatusManager, segments.getInvalidSegments());
+    splits.addAll(batchSplits);
 
     // add all splits of streaming
     List<InputSplit> splitsOfStreaming = getSplitsOfStreaming(job, streamSegments, carbonTable);
diff --git a/integration/flink/src/test/scala/org/apache/carbon/flink/TestCarbonWriter.scala b/integration/flink/src/test/scala/org/apache/carbon/flink/TestCarbonWriter.scala
index 72625dc..9edf7b9 100644
--- a/integration/flink/src/test/scala/org/apache/carbon/flink/TestCarbonWriter.scala
+++ b/integration/flink/src/test/scala/org/apache/carbon/flink/TestCarbonWriter.scala
@@ -28,6 +28,7 @@
 import org.apache.spark.sql.test.util.QueryTest
 
 import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.carbondata.core.util.path.CarbonTablePath
 
 class TestCarbonWriter extends QueryTest {
@@ -95,9 +96,20 @@
           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")
+      checkAnswer(sql(s"select count(*) from $tableName"), Seq(Row(1000)))
+      sql(s"select * from $tableName limit 10").show
+      checkAnswer(sql(s"select max(intField) from $tableName"), Seq(Row(999)))
+      checkAnswer(sql(s"select count(intField) from $tableName where intField >= 900"), Seq(Row(100)))
+      CarbonProperties.getInstance().addProperty(CarbonCommonConstants.CARBON_QUERY_STAGE_INPUT, "false")
+
       sql(s"INSERT INTO $tableName STAGE")
 
-      checkAnswer(sql(s"SELECT count(1) FROM $tableName"), Seq(Row(1000)))
+      checkAnswer(sql(s"SELECT count(*) FROM $tableName"), Seq(Row(1000)))
+      checkAnswer(sql(s"select count(intField) from $tableName where intField >= 900"), Seq(Row(100)))
 
       // ensure the stage snapshot file and all stage files are deleted
       assertResult(false)(FileFactory.isFileExist(CarbonTablePath.getStageSnapshotFile(tablePath)))
diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala
index e4c1620..7841131 100644
--- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala
+++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonCountStar.scala
@@ -36,9 +36,11 @@
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil
-import org.apache.carbondata.core.util.ThreadLocalSessionInfo
+import org.apache.carbondata.core.statusmanager.StageInputCollector
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
 import org.apache.carbondata.hadoop.api.{CarbonInputFormat, CarbonTableInputFormat}
 import org.apache.carbondata.hadoop.util.CarbonInputFormatUtil
+import org.apache.carbondata.spark.load.DataLoadProcessBuilderOnSpark
 
 case class CarbonCountStar(
     attributesRaw: Seq[Attribute],
@@ -54,7 +56,7 @@
     CarbonInputFormat.setQuerySegment(job.getConfiguration, carbonTable)
 
     // get row count
-    val rowCount = CarbonUpdateUtil.getRowCount(
+    var rowCount = CarbonUpdateUtil.getRowCount(
       tableInputFormat.getBlockRowCount(
         job,
         carbonTable,
@@ -65,6 +67,17 @@
             carbonTable.getTableName,
             Some(carbonTable.getDatabaseName))).map(_.asJava).orNull, false),
       carbonTable)
+
+    if (CarbonProperties.isQueryStageInputEnabled) {
+      // check for number of row for stage input
+      val splits = StageInputCollector.createInputSplits(carbonTable, job.getConfiguration)
+      if (!splits.isEmpty) {
+        val df = DataLoadProcessBuilderOnSpark.createInputDataFrame(
+          sparkSession, carbonTable, splits.asScala)
+        rowCount += df.count()
+      }
+    }
+
     val valueRaw =
       attributesRaw.head.dataType match {
         case StringType => Seq(UTF8String.fromString(Long.box(rowCount).toString)).toArray