[GitHub] [carbondata] jackylk commented on a change in pull request #3625: [CARBONDATA-3705] Support create and load MV for spark datasource table

2020-02-25 Thread GitBox
jackylk commented on a change in pull request #3625: [CARBONDATA-3705] Support 
create and load MV for spark datasource table
URL: https://github.com/apache/carbondata/pull/3625#discussion_r383712152
 
 

 ##
 File path: 
mv/core/src/main/scala/org/apache/carbondata/mv/extension/MVHelper.scala
 ##
 @@ -220,55 +209,55 @@ object MVHelper {
   } else {
 Seq()
   }
-  partitionerFields = getPartitionerFields(parentPartitionColumns, 
fieldRelationMap)
+  getPartitionerFields(parentPartitionColumns, fieldRelationMap)
+} else {
+  Seq.empty
 }
 
-var order = 0
 val columnOrderMap = new java.util.HashMap[Integer, String]()
 if (partitionerFields.nonEmpty) {
-  fields.foreach { field =>
-columnOrderMap.put(order, field.column)
-order += 1
+  fields.zipWithIndex.foreach { case (field, index) =>
+columnOrderMap.put(index, field.column)
   }
 }
 
-// TODO Use a proper DB
-val tableIdentifier = TableIdentifier(
-  dataMapSchema.getDataMapName + "_table", 
selectTables.head.identifier.database)
+val mvTableIdentifier = TableIdentifier(
+  dataMapSchema.getDataMapName + "_table", 
mainTables.head.identifier.database)
+
 // prepare table model of the collected tokens
-val tableModel: TableModel = CarbonParserUtil.prepareTableModel(
+val mvTableModel: TableModel = CarbonParserUtil.prepareTableModel(
   ifNotExistPresent = ifNotExistsSet,
-  CarbonParserUtil.convertDbNameToLowerCase(tableIdentifier.database),
-  tableIdentifier.table.toLowerCase,
+  CarbonParserUtil.convertDbNameToLowerCase(mvTableIdentifier.database),
+  mvTableIdentifier.table.toLowerCase,
   fields,
   partitionerFields,
   tableProperties,
   None,
   isAlterFlow = false,
   None)
 
-val tablePath = if (dmProperties.contains("path")) {
-  dmProperties("path")
+val mvTablePath = if (properties.contains("path")) {
+  properties("path")
 } else {
-  CarbonEnv.getTablePath(tableModel.databaseNameOp, 
tableModel.tableName)(sparkSession)
+  CarbonEnv.getTablePath(mvTableModel.databaseNameOp, 
mvTableModel.tableName)(sparkSession)
 }
-CarbonCreateTableCommand(TableNewProcessor(tableModel),
-  tableModel.ifNotExistsSet, Some(tablePath), isVisible = 
false).run(sparkSession)
+CarbonCreateTableCommand(TableNewProcessor(mvTableModel),
 
 Review comment:
   Do you mean providing sort_columns property when creating the MV? If user 
put sort_columns property in parent table (parquet table),  carbon MV should 
not inherit it, 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:
us...@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

2020-02-25 Thread GitBox
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:
us...@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

2020-02-25 Thread GitBox
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:
us...@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

2020-02-25 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3637: [WIP][CARBONDATA-3721] Support Bucket Table

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3637: [WIP][CARBONDATA-3721] Support Bucket 
Table
URL: https://github.com/apache/carbondata/pull/3637#issuecomment-590739256
 
 
   Build Success with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/456/
   


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:
us...@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

2020-02-25 Thread GitBox
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:
us...@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

2020-02-25 Thread GitBox
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:
us...@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

2020-02-25 Thread GitBox
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:
us...@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

2020-02-25 Thread GitBox
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:
us...@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

2020-02-25 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3625: [CARBONDATA-3705] Support create and load MV for spark datasource table

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3625: [CARBONDATA-3705] Support create and 
load MV for spark datasource table
URL: https://github.com/apache/carbondata/pull/3625#issuecomment-590743314
 
 
   Build Success with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/457/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] akashrn5 commented on a change in pull request #3625: [CARBONDATA-3705] Support create and load MV for spark datasource table

2020-02-25 Thread GitBox
akashrn5 commented on a change in pull request #3625: [CARBONDATA-3705] Support 
create and load MV for spark datasource table
URL: https://github.com/apache/carbondata/pull/3625#discussion_r383722655
 
 

 ##
 File path: 
mv/core/src/main/scala/org/apache/carbondata/mv/extension/MVHelper.scala
 ##
 @@ -220,55 +209,55 @@ object MVHelper {
   } else {
 Seq()
   }
-  partitionerFields = getPartitionerFields(parentPartitionColumns, 
fieldRelationMap)
+  getPartitionerFields(parentPartitionColumns, fieldRelationMap)
+} else {
+  Seq.empty
 }
 
-var order = 0
 val columnOrderMap = new java.util.HashMap[Integer, String]()
 if (partitionerFields.nonEmpty) {
-  fields.foreach { field =>
-columnOrderMap.put(order, field.column)
-order += 1
+  fields.zipWithIndex.foreach { case (field, index) =>
+columnOrderMap.put(index, field.column)
   }
 }
 
-// TODO Use a proper DB
-val tableIdentifier = TableIdentifier(
-  dataMapSchema.getDataMapName + "_table", 
selectTables.head.identifier.database)
+val mvTableIdentifier = TableIdentifier(
+  dataMapSchema.getDataMapName + "_table", 
mainTables.head.identifier.database)
+
 // prepare table model of the collected tokens
-val tableModel: TableModel = CarbonParserUtil.prepareTableModel(
+val mvTableModel: TableModel = CarbonParserUtil.prepareTableModel(
   ifNotExistPresent = ifNotExistsSet,
-  CarbonParserUtil.convertDbNameToLowerCase(tableIdentifier.database),
-  tableIdentifier.table.toLowerCase,
+  CarbonParserUtil.convertDbNameToLowerCase(mvTableIdentifier.database),
+  mvTableIdentifier.table.toLowerCase,
   fields,
   partitionerFields,
   tableProperties,
   None,
   isAlterFlow = false,
   None)
 
-val tablePath = if (dmProperties.contains("path")) {
-  dmProperties("path")
+val mvTablePath = if (properties.contains("path")) {
+  properties("path")
 } else {
-  CarbonEnv.getTablePath(tableModel.databaseNameOp, 
tableModel.tableName)(sparkSession)
+  CarbonEnv.getTablePath(mvTableModel.databaseNameOp, 
mvTableModel.tableName)(sparkSession)
 }
-CarbonCreateTableCommand(TableNewProcessor(tableModel),
-  tableModel.ifNotExistsSet, Some(tablePath), isVisible = 
false).run(sparkSession)
+CarbonCreateTableCommand(TableNewProcessor(mvTableModel),
 
 Review comment:
   yes


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index command from DataMap command

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index 
command from DataMap command
URL: https://github.com/apache/carbondata/pull/3614#issuecomment-590746991
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2155/
   


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:
us...@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

2020-02-25 Thread GitBox
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 createInputSplits(CarbonTable table, 
Configuration hadoopConf)
+  throws ExecutionException, InterruptedException {
+List stageInputFiles = new LinkedList<>();
+List 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 stageInputList, List 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:
us...@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

2020-02-25 Thread GitBox
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 createInputSplits(CarbonTable table, 
Configuration hadoopConf)
+  throws ExecutionException, InterruptedException {
+List stageInputFiles = new LinkedList<>();
+List 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:
us...@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

2020-02-25 Thread GitBox
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:
us...@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

2020-02-25 Thread GitBox
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:
us...@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

2020-02-25 Thread GitBox
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:
us...@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

2020-02-25 Thread GitBox
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:
us...@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

2020-02-25 Thread GitBox
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 createInputSplits(CarbonTable table, 
Configuration hadoopConf)
+  throws ExecutionException, InterruptedException {
+List stageInputFiles = new LinkedList<>();
+List 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 stageInputList, List 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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3637: [WIP][CARBONDATA-3721] Support Bucket Table

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3637: [WIP][CARBONDATA-3721] Support Bucket 
Table
URL: https://github.com/apache/carbondata/pull/3637#issuecomment-590753572
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2157/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-02-25 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] Zhangshunyu commented on issue #3637: [WIP][CARBONDATA-3721] Support Bucket Table

2020-02-25 Thread GitBox
Zhangshunyu commented on issue #3637: [WIP][CARBONDATA-3721] Support Bucket 
Table
URL: https://github.com/apache/carbondata/pull/3637#issuecomment-590775160
 
 
   retest this please


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3625: [CARBONDATA-3705] Support create and load MV for spark datasource table

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3625: [CARBONDATA-3705] Support create and 
load MV for spark datasource table
URL: https://github.com/apache/carbondata/pull/3625#issuecomment-590777086
 
 
   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2158/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3637: [WIP][CARBONDATA-3721] Support Bucket Table

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3637: [WIP][CARBONDATA-3721] Support Bucket 
Table
URL: https://github.com/apache/carbondata/pull/3637#issuecomment-590784776
 
 
   Build Success with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/459/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index command from DataMap command

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index 
command from DataMap command
URL: https://github.com/apache/carbondata/pull/3614#issuecomment-590784619
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/460/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3638: [WIP] Avoid data copy in Writer

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3638: [WIP] Avoid data copy in Writer
URL: https://github.com/apache/carbondata/pull/3638#issuecomment-590786470
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2162/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3638: [WIP] Avoid data copy in Writer

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3638: [WIP] Avoid data copy in Writer
URL: https://github.com/apache/carbondata/pull/3638#issuecomment-590788006
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/461/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3638: [WIP] Avoid data copy in Writer

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3638: [WIP] Avoid data copy in Writer
URL: https://github.com/apache/carbondata/pull/3638#issuecomment-590791016
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2164/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-02-25 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index command from DataMap command

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index 
command from DataMap command
URL: https://github.com/apache/carbondata/pull/3614#issuecomment-590796162
 
 
   Build Success with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/462/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index command from DataMap command

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index 
command from DataMap command
URL: https://github.com/apache/carbondata/pull/3614#issuecomment-590796621
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2161/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3638: [WIP] Avoid data copy in Writer

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3638: [WIP] Avoid data copy in Writer
URL: https://github.com/apache/carbondata/pull/3638#issuecomment-590797805
 
 
   Build Success with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/463/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-590821070
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/464/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index command from DataMap command

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index 
command from DataMap command
URL: https://github.com/apache/carbondata/pull/3614#issuecomment-590821296
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2163/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3637: [WIP][CARBONDATA-3721] Support Bucket Table

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3637: [WIP][CARBONDATA-3721] Support Bucket 
Table
URL: https://github.com/apache/carbondata/pull/3637#issuecomment-590821550
 
 
   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2160/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-590825150
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2165/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] akkio-97 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
akkio-97 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-590825907
 
 
   retest this please


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] QiangCai commented on issue #3625: [CARBONDATA-3705] Support create and load MV for spark datasource table

2020-02-25 Thread GitBox
QiangCai commented on issue #3625: [CARBONDATA-3705] Support create and load MV 
for spark datasource table
URL: https://github.com/apache/carbondata/pull/3625#issuecomment-590825798
 
 
   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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] asfgit closed pull request #3625: [CARBONDATA-3705] Support create and load MV for spark datasource table

2020-02-25 Thread GitBox
asfgit closed pull request #3625: [CARBONDATA-3705] Support create and load MV 
for spark datasource table
URL: https://github.com/apache/carbondata/pull/3625
 
 
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-590830243
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2166/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-590833816
 
 
   Build Success with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/467/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-590839168
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2167/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-590848596
 
 
   Build Success with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/468/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-02-25 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


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

2020-02-25 Thread GitBox
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:
us...@infra.apache.org


With regards,
Apache Git Services


[jira] [Resolved] (CARBONDATA-3710) Support query of stage files

2020-02-25 Thread Ajantha Bhat (Jira)


 [ 
https://issues.apache.org/jira/browse/CARBONDATA-3710?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Ajantha Bhat resolved CARBONDATA-3710.
--
Fix Version/s: 2.0.0
   Resolution: Fixed

> Support query of stage files
> 
>
> Key: CARBONDATA-3710
> URL: https://issues.apache.org/jira/browse/CARBONDATA-3710
> Project: CarbonData
>  Issue Type: New Feature
>Reporter: Jacky Li
>Priority: Major
> Fix For: 2.0.0
>
>  Time Spent: 5.5h
>  Remaining Estimate: 0h
>
> 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.
> We should add a feature to make stage files queryable before insert command.
> Since these stage files do not have index, they need to be scanned like 
> normal columnar files



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index command from DataMap command

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index 
command from DataMap command
URL: https://github.com/apache/carbondata/pull/3614#issuecomment-590890828
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/469/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-590895701
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/470/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-590911308
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/471/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3584: [WIP] Support SegmentLevel MinMax for better Pruning and less driver memory usage for cache

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3584: [WIP] Support SegmentLevel MinMax for 
better Pruning and less driver memory usage for cache
URL: https://github.com/apache/carbondata/pull/3584#issuecomment-590918708
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/472/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index command from DataMap command

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index 
command from DataMap command
URL: https://github.com/apache/carbondata/pull/3614#issuecomment-590921139
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/473/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] jackylk commented on issue #3615: [CARBONDATA-3637] Use optimized insert flow for MV and insert stage command

2020-02-25 Thread GitBox
jackylk commented on issue #3615: [CARBONDATA-3637] Use optimized insert flow 
for MV and insert stage command
URL: https://github.com/apache/carbondata/pull/3615#issuecomment-590921776
 
 
   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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] asfgit closed pull request #3615: [CARBONDATA-3637] Use optimized insert flow for MV and insert stage command

2020-02-25 Thread GitBox
asfgit closed pull request #3615: [CARBONDATA-3637] Use optimized insert flow 
for MV and insert stage command
URL: https://github.com/apache/carbondata/pull/3615
 
 
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-590926532
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/474/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] jackylk commented on issue #3634: [CARBONDATA-3720] Support alter table scenario for new insert into flow

2020-02-25 Thread GitBox
jackylk commented on issue #3634: [CARBONDATA-3720] Support alter table 
scenario for new insert into flow
URL: https://github.com/apache/carbondata/pull/3634#issuecomment-590928581
 
 
   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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index command from DataMap command

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index 
command from DataMap command
URL: https://github.com/apache/carbondata/pull/3614#issuecomment-590929249
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2168/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[jira] [Resolved] (CARBONDATA-3637) Improve insert into performance and decrease memory foot print

2020-02-25 Thread Jacky Li (Jira)


 [ 
https://issues.apache.org/jira/browse/CARBONDATA-3637?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jacky Li resolved CARBONDATA-3637.
--
Fix Version/s: 2.0.0
   Resolution: Fixed

> Improve insert into performance and decrease memory foot print
> --
>
> Key: CARBONDATA-3637
> URL: https://issues.apache.org/jira/browse/CARBONDATA-3637
> Project: CarbonData
>  Issue Type: Improvement
>Reporter: Ajantha Bhat
>Assignee: Ajantha Bhat
>Priority: Major
> Fix For: 2.0.0
>
>  Time Spent: 29h 20m
>  Remaining Estimate: 0h
>
> Currently carbondata "insert into" uses the CarbonLoadDataCommand itself.
> Load process has steps like parsing and converter step with bad record
> support.
> Insert into doesn't require these steps as data is already validated and
> converted from source table or dataframe.
> Some identified changes are below.
> 1. Need to refactor and separate load and insert at driver side to skip
> converter step and unify flow for No sort and global sort insert.
> 2. Need to avoid reorder of each row. By changing select dataframe's
> projection order itself during the insert into.
> 3. For carbon to carbon insert, need to provide the ReadSupport and use
> RecordReader (vector reader currently doesn't support ReadSupport) to
> handle null values, time stamp cutoff (direct dictionary) from scanRDD
> result.
> 4. Need to handle insert into partition/non-partition table in local sort,
> global sort, no sort, range columns, compaction flow.
> The final goal is to improve insert performance by keeping only required
> logic and also decrease the memory footprint.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[jira] [Resolved] (CARBONDATA-3720) Support new optimized insert flow for Alter table scenarios

2020-02-25 Thread Jacky Li (Jira)


 [ 
https://issues.apache.org/jira/browse/CARBONDATA-3720?page=com.atlassian.jira.plugin.system.issuetabpanels:all-tabpanel
 ]

Jacky Li resolved CARBONDATA-3720.
--
Resolution: Fixed

> Support new optimized insert flow for Alter table scenarios
> ---
>
> Key: CARBONDATA-3720
> URL: https://issues.apache.org/jira/browse/CARBONDATA-3720
> Project: CarbonData
>  Issue Type: Sub-task
>Reporter: Ajantha Bhat
>Assignee: Ajantha Bhat
>Priority: Major
> Fix For: 2.0.0
>
>  Time Spent: 2h 10m
>  Remaining Estimate: 0h
>
> Currently, the rearrange logic is based on schema ordinal.
> For alter table drop and add columns, schema ordinal based re arrange may not 
> work as index will be outside the projection size.
> Hence don't use schema ordinal for rerrange, implemented a position map based 
> rearrange.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [carbondata] asfgit closed pull request #3634: [CARBONDATA-3720] Support alter table scenario for new insert into flow

2020-02-25 Thread GitBox
asfgit closed pull request #3634: [CARBONDATA-3720] Support alter table 
scenario for new insert into flow
URL: https://github.com/apache/carbondata/pull/3634
 
 
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index command from DataMap command

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index 
command from DataMap command
URL: https://github.com/apache/carbondata/pull/3614#issuecomment-590940395
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2172/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on 
partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-590942397
 
 
   Can one of the admins verify this patch?


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] maheshrajus opened a new pull request #3639: [WIP] Secondary Index enable on partition Table

2020-02-25 Thread GitBox
maheshrajus opened a new pull request #3639: [WIP] Secondary Index enable on 
partition Table
URL: https://github.com/apache/carbondata/pull/3639
 
 
### Why is this PR needed?

[WIP] Secondary Index enable on partition Table
### What changes were proposed in this PR?
   
   
### Does this PR introduce any user interface change?
- No
- Yes. (please explain the change and update document)
   
### Is any new testcase added?
- No
- Yes
   
   
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on partition Table

2020-02-25 Thread GitBox
ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on 
partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-590949550
 
 
   add to whitelist


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on partition Table

2020-02-25 Thread GitBox
ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on 
partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-590949644
 
 
   retest this please


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3584: [WIP] Support SegmentLevel MinMax for better Pruning and less driver memory usage for cache

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3584: [WIP] Support SegmentLevel MinMax for 
better Pruning and less driver memory usage for cache
URL: https://github.com/apache/carbondata/pull/3584#issuecomment-590959104
 
 
   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2171/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index command from DataMap command

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index 
command from DataMap command
URL: https://github.com/apache/carbondata/pull/3614#issuecomment-590965050
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/475/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-590968694
 
 
   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2173/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index command from DataMap command

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index 
command from DataMap command
URL: https://github.com/apache/carbondata/pull/3614#issuecomment-591187927
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/476/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[jira] [Created] (CARBONDATA-3722) Create filterExecuter for each segment instead of blocklet, To improve prune performance

2020-02-25 Thread Xingjun Hao (Jira)
Xingjun Hao created CARBONDATA-3722:
---

 Summary: Create filterExecuter for each segment instead of 
blocklet, To improve prune performance
 Key: CARBONDATA-3722
 URL: https://issues.apache.org/jira/browse/CARBONDATA-3722
 Project: CarbonData
  Issue Type: Improvement
Reporter: Xingjun Hao


In the prunning, It will create filterexecuter for each blocklet, which 
involves a huge performance degrade when there are serveral million blocklet. 

We shall create filterexecuter per segment instead of that per blocklet.



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index command from DataMap command

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3614: [CARBONDATA-3693] Separate Index 
command from DataMap command
URL: https://github.com/apache/carbondata/pull/3614#issuecomment-591199786
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2175/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3638: [WIP] Avoid data copy in Writer

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3638: [WIP] Avoid data copy in Writer
URL: https://github.com/apache/carbondata/pull/3638#issuecomment-591226481
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/477/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3638: [WIP] Avoid data copy in Writer

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3638: [WIP] Avoid data copy in Writer
URL: https://github.com/apache/carbondata/pull/3638#issuecomment-591237646
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2176/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3584: [CARBONDATA-3718] Support SegmentLevel MinMax for better Pruning and less driver memory usage for cache

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3584: [CARBONDATA-3718] Support SegmentLevel 
MinMax for better Pruning and less driver memory usage for cache
URL: https://github.com/apache/carbondata/pull/3584#issuecomment-591240348
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/478/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3584: [CARBONDATA-3718] Support SegmentLevel MinMax for better Pruning and less driver memory usage for cache

2020-02-25 Thread GitBox
Indhumathi27 commented on a change in pull request #3584: [CARBONDATA-3718] 
Support SegmentLevel MinMax for better Pruning and less driver memory usage for 
cache
URL: https://github.com/apache/carbondata/pull/3584#discussion_r384276547
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
 ##
 @@ -2333,4 +2333,9 @@ private CarbonCommonConstants() {
* Default first day of week
*/
   public static final String CARBON_TIMESERIES_FIRST_DAY_OF_WEEK_DEFAULT = 
"SUNDAY";
+
+  public static final String CARBON_LOAD_ALL_INDEX_TO_CACHE = 
"carbon.load.all.indexes.to.cache";
+
+  public static final String CARBON_LOAD_ALL_INDEX_TO_CACHE_DEFAULT = "true";
 
 Review comment:
   done


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3584: [CARBONDATA-3718] Support SegmentLevel MinMax for better Pruning and less driver memory usage for cache

2020-02-25 Thread GitBox
Indhumathi27 commented on a change in pull request #3584: [CARBONDATA-3718] 
Support SegmentLevel MinMax for better Pruning and less driver memory usage for 
cache
URL: https://github.com/apache/carbondata/pull/3584#discussion_r384276578
 
 

 ##
 File path: core/src/main/java/org/apache/carbondata/core/datamap/Segment.java
 ##
 @@ -85,6 +86,8 @@
*/
   private transient Map options;
 
+  private List segmentMinMax;
 
 Review comment:
   done


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3584: [CARBONDATA-3718] Support SegmentLevel MinMax for better Pruning and less driver memory usage for cache

2020-02-25 Thread GitBox
Indhumathi27 commented on a change in pull request #3584: [CARBONDATA-3718] 
Support SegmentLevel MinMax for better Pruning and less driver memory usage for 
cache
URL: https://github.com/apache/carbondata/pull/3584#discussion_r384276767
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
 ##
 @@ -1265,6 +1292,21 @@ void addPath(String path, FolderDetails details) {
 public void setOptions(Map options) {
   this.options = options;
 }
+
+public List getSegmentMinMax() {
+  List segmentMinMaxList = null;
+  try {
+segmentMinMaxList =
+(List) 
ObjectSerializationUtil.convertStringToObject(segmentMinMax);
+  } catch (IOException e) {
+LOGGER.error("Error while getting segment minmax");
+  }
+  return segmentMinMaxList;
 
 Review comment:
   done


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3584: [CARBONDATA-3718] Support SegmentLevel MinMax for better Pruning and less driver memory usage for cache

2020-02-25 Thread GitBox
Indhumathi27 commented on a change in pull request #3584: [CARBONDATA-3718] 
Support SegmentLevel MinMax for better Pruning and less driver memory usage for 
cache
URL: https://github.com/apache/carbondata/pull/3584#discussion_r384276676
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/indexstore/blockletindex/BlockletDataMapFactory.java
 ##
 @@ -211,15 +292,22 @@ private void 
getTableBlockUniqueIdentifierWrappers(List partition
 
   public Set 
getTableBlockIndexUniqueIdentifiers(Segment segment)
 
 Review comment:
   done


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3584: [CARBONDATA-3718] Support SegmentLevel MinMax for better Pruning and less driver memory usage for cache

2020-02-25 Thread GitBox
Indhumathi27 commented on a change in pull request #3584: [CARBONDATA-3718] 
Support SegmentLevel MinMax for better Pruning and less driver memory usage for 
cache
URL: https://github.com/apache/carbondata/pull/3584#discussion_r384276713
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
 ##
 @@ -182,9 +184,14 @@ public static String genSegmentFileName(String segmentId, 
String UUID) {
* @param UUID  a UUID string used to construct the segment file name
* @return segment file name
 
 Review comment:
   done


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3584: [CARBONDATA-3718] Support SegmentLevel MinMax for better Pruning and less driver memory usage for cache

2020-02-25 Thread GitBox
Indhumathi27 commented on a change in pull request #3584: [CARBONDATA-3718] 
Support SegmentLevel MinMax for better Pruning and less driver memory usage for 
cache
URL: https://github.com/apache/carbondata/pull/3584#discussion_r384276913
 
 

 ##
 File path: 
core/src/main/java/org/apache/carbondata/core/util/SegmentMinMaxStats.java
 ##
 @@ -0,0 +1,61 @@
+/*
+ * 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.util;
+
+import java.util.ArrayList;
+import java.util.HashMap;
+import java.util.List;
+import java.util.Map;
+
+/**
+ * Holds list of block level min max for each segment
+ */
+public class SegmentMinMaxStats {
+
+  private SegmentMinMaxStats() {
+  }
+
+  public static SegmentMinMaxStats getInstance() {
 
 Review comment:
   done


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on partition Table

2020-02-25 Thread GitBox
ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on 
partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591242095
 
 
   retest this please


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] ajantha-bhat removed a comment on issue #3639: [WIP] Secondary Index enable on partition Table

2020-02-25 Thread GitBox
ajantha-bhat removed a comment on issue #3639: [WIP] Secondary Index enable on 
partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-590949550
 
 
   add to whitelist


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] ajantha-bhat removed a comment on issue #3639: [WIP] Secondary Index enable on partition Table

2020-02-25 Thread GitBox
ajantha-bhat removed a comment on issue #3639: [WIP] Secondary Index enable on 
partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-590949644
 
 
   retest this please


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on partition Table

2020-02-25 Thread GitBox
ajantha-bhat commented on issue #3639: [WIP] Secondary Index enable on 
partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591242060
 
 
   add to whitelist


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3584: [CARBONDATA-3718] Support SegmentLevel MinMax for better Pruning and less driver memory usage for cache

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3584: [CARBONDATA-3718] Support SegmentLevel 
MinMax for better Pruning and less driver memory usage for cache
URL: https://github.com/apache/carbondata/pull/3584#issuecomment-591243032
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/480/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on 
partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591242992
 
 
   Build Failed  with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/479/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on 
partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591243583
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2178/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-591250971
 
 
   Build Success with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/481/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] Indhumathi27 commented on issue #3584: [CARBONDATA-3718] Support SegmentLevel MinMax for better Pruning and less driver memory usage for cache

2020-02-25 Thread GitBox
Indhumathi27 commented on issue #3584: [CARBONDATA-3718] Support SegmentLevel 
MinMax for better Pruning and less driver memory usage for cache
URL: https://github.com/apache/carbondata/pull/3584#issuecomment-591252315
 
 
   retest this please


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #3637: [CARBONDATA-3721][CARBONDATA-3590] Support Bucket Table

2020-02-25 Thread GitBox
Indhumathi27 commented on a change in pull request #3637: 
[CARBONDATA-3721][CARBONDATA-3590] Support Bucket Table
URL: https://github.com/apache/carbondata/pull/3637#discussion_r384289659
 
 

 ##
 File path: 
processing/src/main/java/org/apache/carbondata/processing/loading/partition/impl/BucketMurmur3HashPartitionerImpl.java
 ##
 @@ -0,0 +1,181 @@
+/*
+ * 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.processing.loading.partition.impl;
+
+import java.util.List;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.core.datastore.row.CarbonRow;
+import org.apache.carbondata.core.metadata.datatype.DataType;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
+import org.apache.carbondata.core.unsafe.hash.Murmur3_x86_32;
+import org.apache.carbondata.core.unsafe.types.UTF8String;
+import org.apache.carbondata.processing.loading.partition.Partitioner;
+
+/**
+ * Bucket Hash partitioner implementation using Murmur3_x86_32, it keep the 
same hash value as
+ * spark for given input.
+ */
+@InterfaceAudience.Internal
+public class BucketMurmur3HashPartitionerImpl implements 
Partitioner {
+
+  private int numberOfBuckets;
+
+  private Hash[] hashes;
+
+  public BucketMurmur3HashPartitionerImpl(List indexes, 
List columnSchemas,
+  int numberOfBuckets) {
+this.numberOfBuckets = numberOfBuckets;
+hashes = new Hash[indexes.size()];
+for (int i = 0; i < indexes.size(); i++) {
+  DataType dataType = columnSchemas.get(i).getDataType();
+  if (dataType == DataTypes.LONG || dataType == DataTypes.DOUBLE) {
+hashes[i] = new LongHash(indexes.get(i));
+  } else if (dataType == DataTypes.SHORT || dataType == DataTypes.INT ||
+  dataType == DataTypes.FLOAT || dataType == DataTypes.BOOLEAN) {
+hashes[i] = new IntegralHash(indexes.get(i));
+  } else if (DataTypes.isDecimal(dataType)) {
+hashes[i] = new DecimalHash(indexes.get(i));
+  } else if (dataType == DataTypes.TIMESTAMP) {
 
 Review comment:
   What about Hash for Date Type?


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3584: [CARBONDATA-3718] Support SegmentLevel MinMax for better Pruning and less driver memory usage for cache

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3584: [CARBONDATA-3718] Support SegmentLevel 
MinMax for better Pruning and less driver memory usage for cache
URL: https://github.com/apache/carbondata/pull/3584#issuecomment-591257534
 
 
   Build Success with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/482/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3584: [CARBONDATA-3718] Support SegmentLevel MinMax for better Pruning and less driver memory usage for cache

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3584: [CARBONDATA-3718] Support SegmentLevel 
MinMax for better Pruning and less driver memory usage for cache
URL: https://github.com/apache/carbondata/pull/3584#issuecomment-591260435
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2182/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-591261749
 
 
   Build Success with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/483/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] ajantha-bhat opened a new pull request #3640: [WIP] Fix insert failure when partition column is a first column of projection in MV

2020-02-25 Thread GitBox
ajantha-bhat opened a new pull request #3640: [WIP] Fix insert failure when 
partition column is a first column of projection in MV
URL: https://github.com/apache/carbondata/pull/3640
 
 
   
   
### Why is this PR needed?


### What changes were proposed in this PR?
   
   
### Does this PR introduce any user interface change?
- No
- Yes. (please explain the change and update document)
   
### Is any new testcase added?
- No
- Yes
   
   
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on partition Table

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3639: [WIP] Secondary Index enable on 
partition Table
URL: https://github.com/apache/carbondata/pull/3639#issuecomment-591273784
 
 
   Build Success with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/484/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[jira] [Created] (CARBONDATA-3723) insert failure when partition column is a first column of projection in MV

2020-02-25 Thread Ajantha Bhat (Jira)
Ajantha Bhat created CARBONDATA-3723:


 Summary: insert failure when partition column is a first column of 
projection in MV
 Key: CARBONDATA-3723
 URL: https://issues.apache.org/jira/browse/CARBONDATA-3723
 Project: CarbonData
  Issue Type: Bug
Reporter: Ajantha Bhat
 Fix For: 2.0.0


For Alter table drop and add column scenarios, partition columns may not be in 
the end of create-order column. So, added a logic to keep the partition column 
in the end in #3634.

 

But partition may not be the last projection column during mv creation. So now, 
create order partition is kept last, but the projection is not last. Hence the 
insert gives wrong data or fails.

 

so, rearrage partition to last in create order columns, only for alter table 
scenarios



--
This message was sent by Atlassian Jira
(v8.3.4#803005)


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT failures

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3629: [CARBONDATA-3716] Fixed spark 2.4 UT 
failures 
URL: https://github.com/apache/carbondata/pull/3629#issuecomment-591275430
 
 
   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/2183/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] CarbonDataQA1 commented on issue #3640: [CARBONDATA-3723] Fix insert failure when partition column is a first column of projection in MV

2020-02-25 Thread GitBox
CarbonDataQA1 commented on issue #3640: [CARBONDATA-3723] Fix insert failure 
when partition column is a first column of projection in MV
URL: https://github.com/apache/carbondata/pull/3640#issuecomment-591276805
 
 
   Build Success with Spark 2.4.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.4/485/
   


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] marchpure commented on a change in pull request #3620: [CARBONDATA-3700] Optimize pruning performance when prunning with multi…

2020-02-25 Thread GitBox
marchpure commented on a change in pull request #3620: [CARBONDATA-3700] 
Optimize pruning performance when prunning with multi…
URL: https://github.com/apache/carbondata/pull/3620#discussion_r384321402
 
 

 ##
 File path: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/blockprune/BlockPruneQueryTestCase.scala
 ##
 @@ -18,16 +18,30 @@ package org.apache.carbondata.spark.testsuite.blockprune
 
 import java.io.DataOutputStream
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.spark.sql.Row
 import org.scalatest.BeforeAndAfterAll
 import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.spark.sql.test.util.QueryTest
 
 /**
   * This class contains test cases for block prune query
   */
 class BlockPruneQueryTestCase extends QueryTest with BeforeAndAfterAll {
   val outputPath = s"$resourcesPath/block_prune_test.csv"
+  val MULTI_THREAD_ENABLE_FILES_COUNT = "1";
 
 Review comment:
   resolved


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] [carbondata] marchpure commented on a change in pull request #3620: [CARBONDATA-3700] Optimize pruning performance when prunning with multi…

2020-02-25 Thread GitBox
marchpure commented on a change in pull request #3620: [CARBONDATA-3700] 
Optimize pruning performance when prunning with multi…
URL: https://github.com/apache/carbondata/pull/3620#discussion_r384321352
 
 

 ##
 File path: 
integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/blockprune/BlockPruneQueryTestCase.scala
 ##
 @@ -18,16 +18,30 @@ package org.apache.carbondata.spark.testsuite.blockprune
 
 import java.io.DataOutputStream
 
+import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.spark.sql.Row
 import org.scalatest.BeforeAndAfterAll
 import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.util.CarbonProperties
 import org.apache.spark.sql.test.util.QueryTest
 
 /**
   * This class contains test cases for block prune query
   */
 class BlockPruneQueryTestCase extends QueryTest with BeforeAndAfterAll {
   val outputPath = s"$resourcesPath/block_prune_test.csv"
+  val MULTI_THREAD_ENABLE_FILES_COUNT = "1";
+  val MULTI_THREAD_DISABLE_FILES_COUNT
+= 
CarbonCommonConstants.CARBON_DRIVER_PRUNING_MULTI_THREAD_ENABLE_FILES_COUNT_DEFAULT;
+
+  def perpareCarbonProperty(propertyName:String,
+propertyValue:String): Unit ={
+val properties = CarbonProperties.getInstance()
+properties.removeProperty(propertyName)
 
 Review comment:
   resolved


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:
us...@infra.apache.org


With regards,
Apache Git Services


  1   2   >