[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


ajantha-bhat commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534852698



##
File path: 
core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
##
@@ -482,176 +482,6 @@ public boolean accept(CarbonFile file) {
 
   }
 
-  /**
-   * Handling of the clean up of old carbondata files, index files , delete 
delta,
-   * update status files.
-   * @param table clean up will be handled on this table.
-   * @param forceDelete if true then max query execution timeout will not be 
considered.
-   */
-  public static void cleanUpDeltaFiles(CarbonTable table, boolean forceDelete) 
throws IOException {

Review comment:
   I didn't see delta file cleanup from the trashManager, will there be any 
impact because of this ? we have to test with stale delta files once before 
removing I think.





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




[GitHub] [carbondata] Karan980 commented on pull request #4017: [CARBONDATA-4022] Fix invalid path issue for segment added through alter table add segment query.

2020-12-02 Thread GitBox


Karan980 commented on pull request #4017:
URL: https://github.com/apache/carbondata/pull/4017#issuecomment-737728408


   > update the description with the actual root cause and updated solution
   
   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




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


ajantha-bhat commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534825881



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/CarbonTruncateCommand.scala
##
@@ -18,39 +18,31 @@
 package org.apache.spark.sql.execution.command.mutation
 
 import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
-import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
-import org.apache.spark.sql.execution.command.{DataCommand, 
TruncateTableCommand}
-import 
org.apache.spark.sql.execution.command.management.CarbonCleanFilesCommand
-import org.apache.spark.sql.hive.CarbonRelation
+import org.apache.spark.sql.execution.command.{Checker, DataCommand, 
TruncateTableCommand}
 
 import 
org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
 import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
 
 case class CarbonTruncateCommand(child: TruncateTableCommand) extends 
DataCommand {
   override def processData(sparkSession: SparkSession): Seq[Row] = {
 val LOGGER = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
-val dbName = 
CarbonEnv.getDatabaseName(child.tableName.database)(sparkSession)
-val tableName = child.tableName.table
-setAuditTable(dbName, tableName)
-val relation = CarbonEnv.getInstance(sparkSession).carbonMetaStore
-  .lookupRelation(Option(dbName), 
tableName)(sparkSession).asInstanceOf[CarbonRelation]
-if (relation == null) {
-  throw new NoSuchTableException(dbName, tableName)
-}
-if (null == relation.carbonTable) {
-  LOGGER.error(s"Truncate table failed. table not found: 
$dbName.$child.tableName.table")
-  throw new NoSuchTableException(dbName, child.tableName.table)
+Checker.validateTableExists(child.tableName.database, 
child.tableName.table, sparkSession)
+val carbonTable = CarbonEnv.getCarbonTable(
+  child.tableName.database, child.tableName.table)(sparkSession)
+setAuditTable(carbonTable)
+if (!carbonTable.isTransactionalTable) {
+  LOGGER.error(s"Unsupported truncate non-transactional table")
+  throw new MalformedCarbonCommandException(
+"Unsupported truncate non-transactional table")
 }
 if (child.partitionSpec.isDefined) {
   throw new MalformedCarbonCommandException(
 "Unsupported truncate table with specified partition")
 }
-CarbonCleanFilesCommand(
-  databaseNameOp = Option(dbName),
-  tableName = Option(tableName),
-  None,
-  truncateTable = true
-).run(sparkSession)
+

Review comment:
   I didn't see this command in any document. If possible please update it. 
I think this will just empty the table by removing all the data (success, fail, 
stale)





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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress seg

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#issuecomment-737723996


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3280/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4035: [CARBONDATA-4067]: Removing force option in clean files command and changing behaviour when MFD, Compacted and stale Inprogress seg

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#issuecomment-737723330


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5037/
   



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




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


ajantha-bhat commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534784870



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -0,0 +1,112 @@
+/*
+ * 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.trash
+
+import scala.collection.JavaConverters._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, 
LockUsage}
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.{CarbonProperties, CleanFilesUtil, 
TrashUtil}
+
+/**
+ * This object will manage the following data.
+ * 1. .Trash folder
+ * 2. stale segments without metadata
+ * 3. expired segments (MARKED_FOR_DELETE, Compacted, In Progress)
+ */
+object DataTrashManager {
+  private val LOGGER = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  /**
+   * clean garbage data
+   *  1. clean .Trash folder
+   *  2. clean stale segments without metadata
+   *  3. clean expired segments (MARKED_FOR_DELETE, Compacted, In Progress)
+   *
+   * @param carbonTable : CarbonTable Object
+   * @param partitionSpecs : Hive Partitions  details
+   */
+  def cleanGarbageData(
+  carbonTable: CarbonTable,
+  force: Boolean = false,
+  partitionSpecs: Option[Seq[PartitionSpec]] = None): Unit = {
+var carbonCleanFilesLock: ICarbonLock = null
+val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+try {
+  val errorMsg = "Clean files request is failed for " +
+s"${ carbonTable.getQualifiedName }" +
+". Not able to acquire the clean files lock due to another clean files 
" +
+"operation is running in the background."
+  carbonCleanFilesLock = 
CarbonLockUtil.getLockObject(absoluteTableIdentifier,
+LockUsage.CLEAN_FILES_LOCK, errorMsg)
+  // step 1: clean trash folder
+  cleanTrashFolder(carbonTable, force)
+  // step 2: clean stale segments which are not exists in metadata
+  cleanStaleSegments(carbonTable)

Review comment:
   `moveStaleSegmentsToTrash` is the better name for this function ?





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




[GitHub] [carbondata] akashrn5 commented on a change in pull request #4018: [CARBONDATA-4055]Fix creation of empty segment directory and meta entry when there is no update/insert data

2020-12-02 Thread GitBox


akashrn5 commented on a change in pull request #4018:
URL: https://github.com/apache/carbondata/pull/4018#discussion_r534771957



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/CarbonMergeDataSetCommand.scala
##
@@ -212,17 +212,20 @@ case class CarbonMergeDataSetCommand(
 }
 
 val dataFrame = loadDF.select(tableCols.map(col): _*)
-CarbonInsertIntoCommand(databaseNameOp = Some(carbonTable.getDatabaseName),
-  tableName = carbonTable.getTableName,
-  options = Map("fileheader" -> header),
-  isOverwriteTable = false,
-  dataFrame.queryExecution.logical,
-  carbonTable.getTableInfo,
-  Map.empty,
-  Map.empty,
-  new OperationContext,
-  updateTableModel
-).run(sparkSession)
+if (mergeMatches.matchList.flatMap(_.getActions).exists(p => 
p.isInstanceOf[InsertAction]) ||

Review comment:
   handled, please check





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




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


ajantha-bhat commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534771387



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/trash/DataTrashManager.scala
##
@@ -0,0 +1,112 @@
+/*
+ * 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.trash
+
+import scala.collection.JavaConverters._
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.locks.{CarbonLockUtil, ICarbonLock, 
LockUsage}
+import org.apache.carbondata.core.metadata.SegmentFileStore
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+import org.apache.carbondata.core.statusmanager.SegmentStatusManager
+import org.apache.carbondata.core.util.{CarbonProperties, CleanFilesUtil, 
TrashUtil}
+
+/**
+ * This object will manage the following data.
+ * 1. .Trash folder
+ * 2. stale segments without metadata
+ * 3. expired segments (MARKED_FOR_DELETE, Compacted, In Progress)
+ */
+object DataTrashManager {
+  private val LOGGER = 
LogServiceFactory.getLogService(this.getClass.getCanonicalName)
+
+  /**
+   * clean garbage data
+   *  1. clean .Trash folder
+   *  2. clean stale segments without metadata
+   *  3. clean expired segments (MARKED_FOR_DELETE, Compacted, In Progress)
+   *
+   * @param carbonTable : CarbonTable Object
+   * @param partitionSpecs : Hive Partitions  details
+   */
+  def cleanGarbageData(
+  carbonTable: CarbonTable,
+  force: Boolean = false,
+  partitionSpecs: Option[Seq[PartitionSpec]] = None): Unit = {
+var carbonCleanFilesLock: ICarbonLock = null
+val absoluteTableIdentifier = carbonTable.getAbsoluteTableIdentifier
+try {
+  val errorMsg = "Clean files request is failed for " +
+s"${ carbonTable.getQualifiedName }" +
+". Not able to acquire the clean files lock due to another clean files 
" +
+"operation is running in the background."
+  carbonCleanFilesLock = 
CarbonLockUtil.getLockObject(absoluteTableIdentifier,
+LockUsage.CLEAN_FILES_LOCK, errorMsg)
+  // step 1: clean trash folder
+  cleanTrashFolder(carbonTable, force)

Review comment:
   `checkAndCleanTrashFolder`  
   As we dont clean always. Function name give a feeling that we are cleaning 
trash folder here





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




[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534763831



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -439,6 +430,73 @@ public boolean accept(CarbonFile file) {
 return null;
   }
 
+  /**
+   * Get old and invalid files which have already been merged to a mergeindex 
file.In segment folder
+   * we may have both .index files and .mergeindex files, as we are not 
deleting index files
+   * immediately for old tables, this method reads mergeindex file and adds 
mapped index files to a
+   * list and returns.If more than one mergeindex file is present, considers 
the latest one as valid
+   * Ex: We have 3 files in segment. Segment0/ 1.index , 1.mergeindex file, 
1.carbondata.
+   * 1.index is merged to 1.mergeindex. Here it returns merged index file - 
1.index.
+   */
+  public static Set getInvalidAndMergedIndexFiles(List 
indexFiles)
+  throws IOException {
+SegmentIndexFileStore indexFileStore = new SegmentIndexFileStore();

Review comment:
   This method is called during read, when segment/table status file is not 
present or gets deleted, and when writing segment data size in tablestatus 
file. So for all old/new tables this method is called and taken care of. For 
new tables also, when stale data is present, it is used to filter invalid 
files. Ex: SI load when MT has stale index files, as during SI load MT segment 
file/ status file name is not updated, we directly get from segment directory. 





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




[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534745677



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -1044,7 +1045,7 @@ public static void cleanSegments(CarbonTable table,
   Long fileTimestamp = CarbonUpdateUtil.getTimeStampAsLong(indexFile
   .substring(indexFile.lastIndexOf(CarbonCommonConstants.HYPHEN) + 
1,
   indexFile.length() - 
CarbonTablePath.INDEX_FILE_EXT.length()));
-  if (CarbonUpdateUtil.isMaxQueryTimeoutExceeded(fileTimestamp) || 
forceDelete) {
+  if (TrashUtil.isTrashDataTimeout(fileTimestamp) || forceDelete) {

Review comment:
   yes, the requirement just like 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




[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534746030



##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -192,12 +192,9 @@ public static void emptyTrash(String tablePath) {
   }
 
   /**
-   * This will tell whether the trash retention time has expired or not
-   *
-   * @param fileTimestamp
-   * @return
+   * whether trash data inside of .Trash folder is time out

Review comment:
   no, can you check another method? and this method is private and only 
used by trash util to process trash folder





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




[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534746030



##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -192,12 +192,9 @@ public static void emptyTrash(String tablePath) {
   }
 
   /**
-   * This will tell whether the trash retention time has expired or not
-   *
-   * @param fileTimestamp
-   * @return
+   * whether trash data inside of .Trash folder is time out

Review comment:
   no, can you check another method

##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -192,12 +192,9 @@ public static void emptyTrash(String tablePath) {
   }
 
   /**
-   * This will tell whether the trash retention time has expired or not
-   *
-   * @param fileTimestamp
-   * @return
+   * whether trash data inside of .Trash folder is time out

Review comment:
   no, can you check another method?





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




[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534746030



##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -192,12 +192,9 @@ public static void emptyTrash(String tablePath) {
   }
 
   /**
-   * This will tell whether the trash retention time has expired or not
-   *
-   * @param fileTimestamp
-   * @return
+   * whether trash data inside of .Trash folder is time out

Review comment:
   no





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




[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534745677



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -1044,7 +1045,7 @@ public static void cleanSegments(CarbonTable table,
   Long fileTimestamp = CarbonUpdateUtil.getTimeStampAsLong(indexFile
   .substring(indexFile.lastIndexOf(CarbonCommonConstants.HYPHEN) + 
1,
   indexFile.length() - 
CarbonTablePath.INDEX_FILE_EXT.length()));
-  if (CarbonUpdateUtil.isMaxQueryTimeoutExceeded(fileTimestamp) || 
forceDelete) {
+  if (TrashUtil.isTrashDataTimeout(fileTimestamp) || forceDelete) {

Review comment:
   yes, the requirement is just like 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




[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534744818



##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -208,6 +205,14 @@ public static boolean isTrashRetentionTimeoutExceeded(long 
fileTimestamp) {
 return difference > retentionMilliSeconds;
   }
 
+  /**
+   * whether trash data outside of .Trash folder is time out
+   */
+  public static boolean isTrashDataTimeout(long fileTimestamp) {
+return isTrashRetentionTimeoutExceeded(fileTimestamp) &&

Review comment:
   isTrashRetentionTimeoutExceeded:  whether trash data inside of .Trash 
folder is time out
   isTrashDataTimeout: whether trash data outside of .Trash folder is time out
   
   TrashRetentionTimeout can be 0, but MaxQueryTimeout not.





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




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


ajantha-bhat commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534743440



##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -192,12 +192,9 @@ public static void emptyTrash(String tablePath) {
   }
 
   /**
-   * This will tell whether the trash retention time has expired or not
-   *
-   * @param fileTimestamp
-   * @return
+   * whether trash data inside of .Trash folder is time out

Review comment:
   now we are using this for data outside trash also, so change the 
description.
   
   `returns true, if the current timestamp has expired based on 
carbon.trash.retention.days `





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




[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534742148



##
File path: 
core/src/main/java/org/apache/carbondata/core/util/CleanFilesUtil.java
##
@@ -146,8 +146,8 @@ public static void 
cleanStaleSegmentsForPartitionTable(CarbonTable carbonTable)
* in the metadata folder and is not present in the table status file is 
considered as a
* stale segment. Only comparing from tablestatus file, not checking 
tablestatus.history file
*/
-  private static void getStaleSegmentFiles(CarbonTable carbonTable, 
List staleSegmentFiles,
-  List redundantSegmentFile) {
+  private static void collectStaleSegmentFiles(CarbonTable carbonTable,

Review comment:
   this method need return two lists, so it passes two lists in parameters.
   now it return void, so "collect" is better than "get"





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




[GitHub] [carbondata] QiangCai commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


QiangCai commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534742148



##
File path: 
core/src/main/java/org/apache/carbondata/core/util/CleanFilesUtil.java
##
@@ -146,8 +146,8 @@ public static void 
cleanStaleSegmentsForPartitionTable(CarbonTable carbonTable)
* in the metadata folder and is not present in the table status file is 
considered as a
* stale segment. Only comparing from tablestatus file, not checking 
tablestatus.history file
*/
-  private static void getStaleSegmentFiles(CarbonTable carbonTable, 
List staleSegmentFiles,
-  List redundantSegmentFile) {
+  private static void collectStaleSegmentFiles(CarbonTable carbonTable,

Review comment:
   this method need return two lists, so it passes two lists in parameters





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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-737700791


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3279/
   



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




[GitHub] [carbondata] Zhangshunyu commented on pull request #4032: [CARBONDATA-4065] Support MERGE INTO SQL Command

2020-12-02 Thread GitBox


Zhangshunyu commented on pull request #4032:
URL: https://github.com/apache/carbondata/pull/4032#issuecomment-737698494


   added antlr4-maven-plugin to manage the antlr files during compile



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




[GitHub] [carbondata] nihal0107 opened a new pull request #4037: [WIP] Handle Alter long string for SI col and added FTs for SI

2020-12-02 Thread GitBox


nihal0107 opened a new pull request #4037:
URL: https://github.com/apache/carbondata/pull/4037


### 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




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


ajantha-bhat commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534730624



##
File path: 
core/src/main/java/org/apache/carbondata/core/util/CleanFilesUtil.java
##
@@ -146,8 +146,8 @@ public static void 
cleanStaleSegmentsForPartitionTable(CarbonTable carbonTable)
* in the metadata folder and is not present in the table status file is 
considered as a
* stale segment. Only comparing from tablestatus file, not checking 
tablestatus.history file
*/
-  private static void getStaleSegmentFiles(CarbonTable carbonTable, 
List staleSegmentFiles,
-  List redundantSegmentFile) {
+  private static void collectStaleSegmentFiles(CarbonTable carbonTable,

Review comment:
   I suggest `getStaleSegmentFiles` is fine, but make a list inside the 
function and send as a return value.  Passing as an argument makes sense only 
when the list has some pre-entries or used in recursion. 





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




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


ajantha-bhat commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534727593



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -1044,7 +1045,7 @@ public static void cleanSegments(CarbonTable table,
   Long fileTimestamp = CarbonUpdateUtil.getTimeStampAsLong(indexFile
   .substring(indexFile.lastIndexOf(CarbonCommonConstants.HYPHEN) + 
1,
   indexFile.length() - 
CarbonTablePath.INDEX_FILE_EXT.length()));
-  if (CarbonUpdateUtil.isMaxQueryTimeoutExceeded(fileTimestamp) || 
forceDelete) {
+  if (TrashUtil.isTrashDataTimeout(fileTimestamp) || forceDelete) {

Review comment:
   Also update this behavior change in document, now 
`max.query.execution.time`  expire will not delete the data but trash time has 
to expire also. 





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-737695440


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5036/
   



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




[GitHub] [carbondata] kunal642 commented on a change in pull request #4017: [CARBONDATA-4022] Fix invalid path issue for segment added through alter table add segment query.

2020-12-02 Thread GitBox


kunal642 commented on a change in pull request #4017:
URL: https://github.com/apache/carbondata/pull/4017#discussion_r534686282



##
File path: 
core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
##
@@ -219,7 +220,12 @@ public void deserializeFields(DataInput in, String[] 
locations, String tablePath
 if (in.readBoolean()) {
   indexUniqueId = in.readUTF();
 }
-setFilePath(tablePath + getPath());
+String filePath = getPath();
+if (filePath.startsWith(File.separator)) {

Review comment:
   better to change this check to 
   ```
   if (filePath.contains(tablePath)) {
   // don't add table path
   } else {
   // add table path
   }
   ```





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




[GitHub] [carbondata] Zhangshunyu closed pull request #4036: [WIP][Test CI]

2020-12-02 Thread GitBox


Zhangshunyu closed pull request #4036:
URL: https://github.com/apache/carbondata/pull/4036


   



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




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


ajantha-bhat commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534713302



##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -208,6 +205,14 @@ public static boolean isTrashRetentionTimeoutExceeded(long 
fileTimestamp) {
 return difference > retentionMilliSeconds;
   }
 
+  /**
+   * whether trash data outside of .Trash folder is time out
+   */
+  public static boolean isTrashDataTimeout(long fileTimestamp) {
+return isTrashRetentionTimeoutExceeded(fileTimestamp) &&

Review comment:
   In the default case, when `isTrashRetentionTimeoutExceeded`  is true, 
then `isMaxQueryTimeoutExceeded` will always be true. 
   why not remove `queryMaxTimeout` from code and in query flow, clean the 
metadata indexed trash based on `trashRetentionTimeout` ??





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




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


ajantha-bhat commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534713302



##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -208,6 +205,14 @@ public static boolean isTrashRetentionTimeoutExceeded(long 
fileTimestamp) {
 return difference > retentionMilliSeconds;
   }
 
+  /**
+   * whether trash data outside of .Trash folder is time out
+   */
+  public static boolean isTrashDataTimeout(long fileTimestamp) {
+return isTrashRetentionTimeoutExceeded(fileTimestamp) &&

Review comment:
   In the default case, when isTrashRetentionTimeoutExceeded  , 
isMaxQueryTimeoutExceeded will always be true. 
   why not remove `queryMaxTimeout` from code and in query flow, clean the 
metadata indexed trash based on `trashRetentionTimeout` ??





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




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


ajantha-bhat commented on a change in pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#discussion_r534713302



##
File path: core/src/main/java/org/apache/carbondata/core/util/TrashUtil.java
##
@@ -208,6 +205,14 @@ public static boolean isTrashRetentionTimeoutExceeded(long 
fileTimestamp) {
 return difference > retentionMilliSeconds;
   }
 
+  /**
+   * whether trash data outside of .Trash folder is time out
+   */
+  public static boolean isTrashDataTimeout(long fileTimestamp) {
+return isTrashRetentionTimeoutExceeded(fileTimestamp) &&

Review comment:
   In the default case, when isTrashRetentionTimeoutExceeded  , 
isMaxQueryTimeoutExceeded will always be true. 
   why not remove `queryMaxTimeout` and in query flow clean the metadata 
indexed trash based on `trashRetentionTimeout` ??





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




[GitHub] [carbondata] kunal642 commented on a change in pull request #4018: [CARBONDATA-4055]Fix creation of empty segment directory and meta entry when there is no update/insert data

2020-12-02 Thread GitBox


kunal642 commented on a change in pull request #4018:
URL: https://github.com/apache/carbondata/pull/4018#discussion_r534693837



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/CarbonMergeDataSetCommand.scala
##
@@ -212,17 +212,20 @@ case class CarbonMergeDataSetCommand(
 }
 
 val dataFrame = loadDF.select(tableCols.map(col): _*)
-CarbonInsertIntoCommand(databaseNameOp = Some(carbonTable.getDatabaseName),
-  tableName = carbonTable.getTableName,
-  options = Map("fileheader" -> header),
-  isOverwriteTable = false,
-  dataFrame.queryExecution.logical,
-  carbonTable.getTableInfo,
-  Map.empty,
-  Map.empty,
-  new OperationContext,
-  updateTableModel
-).run(sparkSession)
+if (mergeMatches.matchList.flatMap(_.getActions).exists(p => 
p.isInstanceOf[InsertAction]) ||

Review comment:
   Can we move this and the check in CarbonProjectForUpdateCommand.scala to 
a common place in InsertIntoCommand?





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




[GitHub] [carbondata] kunal642 commented on pull request #4017: [CARBONDATA-4022] Fix invalid path issue for segment added through alter table add segment query.

2020-12-02 Thread GitBox


kunal642 commented on pull request #4017:
URL: https://github.com/apache/carbondata/pull/4017#issuecomment-737680164


   update the description with the actual root cause and updated solution



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




[GitHub] [carbondata] kunal642 commented on a change in pull request #4017: [CARBONDATA-4022] Fix invalid path issue for segment added through alter table add segment query.

2020-12-02 Thread GitBox


kunal642 commented on a change in pull request #4017:
URL: https://github.com/apache/carbondata/pull/4017#discussion_r534686282



##
File path: 
core/src/main/java/org/apache/carbondata/core/indexstore/ExtendedBlocklet.java
##
@@ -219,7 +220,12 @@ public void deserializeFields(DataInput in, String[] 
locations, String tablePath
 if (in.readBoolean()) {
   indexUniqueId = in.readUTF();
 }
-setFilePath(tablePath + getPath());
+String filePath = getPath();
+if (filePath.startsWith(File.separator)) {

Review comment:
   better to change this check to 
   ```
   if (filePath.contains(tablePath)) {
   // don't add table path
   } else {
   // add table path
   }
   ```





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




[GitHub] [carbondata] ajantha-bhat commented on a change in pull request #4001: [CARBONDATA-4029] [CARBONDATA-3908] Issue while adding segments through alter add segment command

2020-12-02 Thread GitBox


ajantha-bhat commented on a change in pull request #4001:
URL: https://github.com/apache/carbondata/pull/4001#discussion_r534681054



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -877,6 +877,10 @@ public SegmentFile getSegmentFile() {
 }
 if 
(entry.getValue().status.equals(SegmentStatus.SUCCESS.getMessage())) {
   String mergeFileName = entry.getValue().getMergeFileName();
+  // remove file separator if already present

Review comment:
   if the user is putting "/", while storing itself, can we remove that and 
store it, instead of skipping here ? because some other place if the stored 
value is used. again same code has to be there also





This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

For queries about this service, please contact Infrastructure at:
us...@infra.apache.org




[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534676791



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -162,15 +162,25 @@ public static void writeSegmentFile(String tablePath, 
String segmentId, String t
* corresponding partitions.
*/
   public static void writeSegmentFile(String tablePath, final String taskNo, 
String location,
-  String timeStamp, List partitionNames, boolean isMergeIndexFlow) 
throws IOException {
-String tempFolderLoc = timeStamp + ".tmp";
-String writePath = CarbonTablePath.getSegmentFilesLocation(tablePath) + 
"/" + tempFolderLoc;
+  String timeStamp, List partitionNames, boolean isMergeIndexFlow,
+  boolean readFileFooterFromCarbonDataFile) throws IOException {
+String tempFolderLoc;
+String writePath;
+if (!readFileFooterFromCarbonDataFile) {
+  tempFolderLoc = timeStamp + ".tmp";
+  writePath = CarbonTablePath.getSegmentFilesLocation(tablePath) + "/" + 
tempFolderLoc;
+} else {
+  // If Alter merge index for old tables is triggered,
+  // directly write mergeindex file into segment file location
+  tempFolderLoc = location;
+  writePath = CarbonTablePath.getSegmentFilesLocation(tablePath);
+}
 CarbonFile carbonFile = FileFactory.getCarbonFile(writePath);
 if (!carbonFile.exists()) {
   carbonFile.mkdirs();
 }
 CarbonFile tempFolder;
-if (isMergeIndexFlow) {
+if (isMergeIndexFlow || readFileFooterFromCarbonDataFile) {

Review comment:
   Done. removed unnecessary code from this method.





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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4036: [WIP][Test CI]

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4036:
URL: https://github.com/apache/carbondata/pull/4036#issuecomment-737675578


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3278/
   



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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


vikramahuja1001 commented on a change in pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#discussion_r534678969



##
File path: 
core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##
@@ -173,40 +176,62 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-  boolean isForceDelete) {
-if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == 
oneLoad.getSegmentStatus())
-&& oneLoad.getVisibility().equalsIgnoreCase("true")) {
-  if (isForceDelete) {
-return true;
-  }
-  long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-  return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && 
CarbonUpdateUtil
-  .isMaxQueryTimeoutExceeded(deletionTime);
+  boolean cleanStaleInProgress, boolean cleanCompactedAndMFD) {
+if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+  return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, 
cleanStaleInProgress);
 }
-
 return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails 
oneLoad,
-  boolean isForceDelete) {
+  boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
 // Check if the segment is added externally and path is set then do not 
delete it
-if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-|| SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && 
(oneLoad.getPath() == null
-|| oneLoad.getPath().equalsIgnoreCase("NA"))) {
-  if (isForceDelete) {
-return true;
-  }
-  long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) 
{
+  return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, 
cleanStaleInProgress);
+}
+return false;
+  }
 
-  return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && 
CarbonUpdateUtil
+  private static Boolean checkLoadDeletionLogic(LoadMetadataDetails oneLoad,
+  boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
+/*
+ * if cleanStaleInProgress == false and  cleanCompactedAndMFD == false, 
clean MFD and Compacted
+ *  segments after trashtimeout(7days) && query timeout(1 hr)
+ * if cleanStaleInProgress == false and  cleanCompactedAndMFD == true, 
clean MFD and Compacted
+ *  segments immediately
+ * if cleanStaleInProgress == true and  cleanCompactedAndMFD == false, 
clean Stale Inprogress
+ *  segments after 7 days(taking carbon.trash.retention.time value)
+ * if cleanStaleInProgress == true and  cleanCompactedAndMFD == true, 
clean MFD, Compacted and
+ *  stale inprogress segments immediately.
+ */
+if (!cleanCompactedAndMFD && !cleanStaleInProgress) {
+  if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || 
SegmentStatus
+  .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
+long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && 
CarbonUpdateUtil
   .isMaxQueryTimeoutExceeded(deletionTime);
-
+  }
+  return false;
+} else if (cleanCompactedAndMFD && !cleanStaleInProgress) {
+  return SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || 
SegmentStatus
+.MARKED_FOR_DELETE == oneLoad.getSegmentStatus();
+} else if (!cleanCompactedAndMFD) {

Review comment:
   I have added comments in the code to tell what exactly is happening and 
also added in the PR description. Please refer it once





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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


vikramahuja1001 commented on a change in pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#discussion_r534678759



##
File path: 
core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##
@@ -173,40 +176,62 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-  boolean isForceDelete) {
-if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == 
oneLoad.getSegmentStatus())
-&& oneLoad.getVisibility().equalsIgnoreCase("true")) {
-  if (isForceDelete) {
-return true;
-  }
-  long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-  return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && 
CarbonUpdateUtil
-  .isMaxQueryTimeoutExceeded(deletionTime);
+  boolean cleanStaleInProgress, boolean cleanCompactedAndMFD) {
+if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+  return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, 
cleanStaleInProgress);
 }
-
 return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails 
oneLoad,
-  boolean isForceDelete) {
+  boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
 // Check if the segment is added externally and path is set then do not 
delete it
-if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-|| SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && 
(oneLoad.getPath() == null
-|| oneLoad.getPath().equalsIgnoreCase("NA"))) {
-  if (isForceDelete) {
-return true;
-  }
-  long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) 
{
+  return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, 
cleanStaleInProgress);
+}
+return false;
+  }
 
-  return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && 
CarbonUpdateUtil
+  private static Boolean checkLoadDeletionLogic(LoadMetadataDetails oneLoad,
+  boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
+/*
+ * if cleanStaleInProgress == false and  cleanCompactedAndMFD == false, 
clean MFD and Compacted
+ *  segments after trashtimeout(7days) && query timeout(1 hr)
+ * if cleanStaleInProgress == false and  cleanCompactedAndMFD == true, 
clean MFD and Compacted
+ *  segments immediately
+ * if cleanStaleInProgress == true and  cleanCompactedAndMFD == false, 
clean Stale Inprogress
+ *  segments after 7 days(taking carbon.trash.retention.time value)
+ * if cleanStaleInProgress == true and  cleanCompactedAndMFD == true, 
clean MFD, Compacted and
+ *  stale inprogress segments immediately.
+ */
+if (!cleanCompactedAndMFD && !cleanStaleInProgress) {
+  if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || 
SegmentStatus
+  .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
+long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && 
CarbonUpdateUtil
   .isMaxQueryTimeoutExceeded(deletionTime);
-
+  }
+  return false;
+} else if (cleanCompactedAndMFD && !cleanStaleInProgress) {

Review comment:
   previously forceDelete would delete MFD, Compacted and stale INprogress 
segments immediately. cleanCompactedAndMFD means only delete MFD, 
Compacted(immediately)  and cleanStaleInProgress means only delete 
staleInprogress segments.





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




[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534678076



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/util/SecondaryIndexUtil.scala
##
@@ -226,85 +233,59 @@ object SecondaryIndexUtil {
   validSegmentsToUse.toList.asJava,
   indexCarbonTable)
   }
-  mergedSegments.asScala.map { seg =>
-val file = SegmentFileStore.writeSegmentFile(
-  indexCarbonTable,
-  seg.getLoadName,
-  carbonLoadModel.getFactTimeStamp.toString,
-  null,
-  null)
-val segment = new Segment(seg.getLoadName, file)
-SegmentFileStore.updateTableStatusFile(indexCarbonTable,
-  seg.getLoadName,
-  file,
-  indexCarbonTable.getCarbonTableIdentifier.getTableId,
-  new SegmentFileStore(tablePath, segment.getSegmentFileName))
-segment
-  }
-
-  val statusLock =
-new 
SegmentStatusManager(indexCarbonTable.getAbsoluteTableIdentifier).getTableStatusLock
-  try {
-val retryCount = 
CarbonLockUtil.getLockProperty(CarbonCommonConstants
-  .NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK,
-  
CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK_DEFAULT)
-val maxTimeout = 
CarbonLockUtil.getLockProperty(CarbonCommonConstants
-  .MAX_TIMEOUT_FOR_CONCURRENT_LOCK,
-  CarbonCommonConstants.MAX_TIMEOUT_FOR_CONCURRENT_LOCK_DEFAULT)
-if (statusLock.lockWithRetries(retryCount, maxTimeout)) {
-  val endTime = System.currentTimeMillis()
-  val loadMetadataDetails = SegmentStatusManager
-.readLoadMetadata(indexCarbonTable.getMetadataPath)
-  loadMetadataDetails.foreach(loadMetadataDetail => {
-if (rebuiltSegments.contains(loadMetadataDetail.getLoadName)) {
-  
loadMetadataDetail.setLoadStartTime(carbonLoadModel.getFactTimeStamp)
-  loadMetadataDetail.setLoadEndTime(endTime)
-  CarbonLoaderUtil
-.addDataIndexSizeIntoMetaEntry(loadMetadataDetail,
-  loadMetadataDetail.getLoadName,
-  indexCarbonTable)
-}
-  })
-  SegmentStatusManager
-
.writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(tablePath),
-  loadMetadataDetails)
-} else {
-  throw new RuntimeException(
-"Not able to acquire the lock for table status updation for 
table " + databaseName +
-"." + indexCarbonTable.getTableName)
-}
-  } finally {
-if (statusLock != null) {
-  statusLock.unlock()
-}
-  }
-  // clear the indexSchema cache for the merged segments, as the index 
files and
-  // data files are rewritten after compaction
+  val segmentToLoadStartTimeMap: scala.collection.mutable.Map[String, 
java.lang.Long] =
+scala.collection.mutable.Map()
   if (mergedSegments.size > 0) {

Review comment:
   removed the check.





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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


vikramahuja1001 commented on a change in pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#discussion_r534678032



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -1039,17 +1039,19 @@ private static void 
writeLoadMetadata(AbsoluteTableIdentifier identifier,
 }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, 
CarbonTable carbonTable,
-  AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] 
details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, 
boolean

Review comment:
   Instead of just a force option deleting all MFD, Compacted and stale 
Insert In Progress segments, dividing them into 2 parameters, 
cleanMFDandCompacted and cleanStaleInProgress. CleanMFDAndCompacted parameter 
will decide if MFD and Compacted segments can be deleted and 
cleanStaleInProgress will decide if stale InProgress segments can be deleted. 
After giving these 2 parameters, force can be removed. if the user wants to do 
force deletion, they can set both cleanMFDandCompacted and cleanStaleInProgress 
to true.





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




[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534677732



##
File path: 
integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonTableCompactor.scala
##
@@ -267,15 +269,20 @@ class CarbonTableCompactor(carbonLoadModel: 
CarbonLoadModel,
 // Merge all partition files into a single file.
 segmentFileName =
   mergedLoadNumber + "_" + carbonLoadModel.getFactTimeStamp
-val segmentFile = SegmentFileStore
-  .mergeSegmentFiles(readPath,
-segmentFileName,
-
CarbonTablePath.getSegmentFilesLocation(carbonLoadModel.getTablePath))
-if (segmentFile != null) {
-  SegmentFileStore
-.moveFromTempFolder(segmentFile,
-  carbonLoadModel.getFactTimeStamp + ".tmp",
-  carbonLoadModel.getTablePath)
+if (!isPropertySet(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT,
+  CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT)) {

Review comment:
   Done

##
File path: 
integration/spark/src/main/scala/org/apache/spark/rdd/CarbonMergeFilesRDD.scala
##
@@ -167,6 +173,21 @@ object CarbonMergeFilesRDD {
 executorService.submit(new Runnable {
   override def run(): Unit = {
 ThreadLocalSessionInfo.setCarbonSessionInfo(carbonSessionInfo)
+// If Alter merge index for old tables is triggered, do not 
delete index files
+// immediately to avoid index file not found during concurrent 
queries
+if (readFileFooterFromCarbonDataFile ||
+
!isPropertySet(CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT,
+  
CarbonCommonConstants.CARBON_MERGE_INDEX_IN_SEGMENT_DEFAULT)) {

Review comment:
   Done

##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/util/SecondaryIndexUtil.scala
##
@@ -226,85 +233,59 @@ object SecondaryIndexUtil {
   validSegmentsToUse.toList.asJava,
   indexCarbonTable)
   }
-  mergedSegments.asScala.map { seg =>
-val file = SegmentFileStore.writeSegmentFile(
-  indexCarbonTable,
-  seg.getLoadName,
-  carbonLoadModel.getFactTimeStamp.toString,
-  null,
-  null)
-val segment = new Segment(seg.getLoadName, file)
-SegmentFileStore.updateTableStatusFile(indexCarbonTable,
-  seg.getLoadName,
-  file,
-  indexCarbonTable.getCarbonTableIdentifier.getTableId,
-  new SegmentFileStore(tablePath, segment.getSegmentFileName))
-segment
-  }
-
-  val statusLock =
-new 
SegmentStatusManager(indexCarbonTable.getAbsoluteTableIdentifier).getTableStatusLock
-  try {
-val retryCount = 
CarbonLockUtil.getLockProperty(CarbonCommonConstants
-  .NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK,
-  
CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK_DEFAULT)
-val maxTimeout = 
CarbonLockUtil.getLockProperty(CarbonCommonConstants
-  .MAX_TIMEOUT_FOR_CONCURRENT_LOCK,
-  CarbonCommonConstants.MAX_TIMEOUT_FOR_CONCURRENT_LOCK_DEFAULT)
-if (statusLock.lockWithRetries(retryCount, maxTimeout)) {
-  val endTime = System.currentTimeMillis()
-  val loadMetadataDetails = SegmentStatusManager
-.readLoadMetadata(indexCarbonTable.getMetadataPath)
-  loadMetadataDetails.foreach(loadMetadataDetail => {
-if (rebuiltSegments.contains(loadMetadataDetail.getLoadName)) {
-  
loadMetadataDetail.setLoadStartTime(carbonLoadModel.getFactTimeStamp)
-  loadMetadataDetail.setLoadEndTime(endTime)
-  CarbonLoaderUtil
-.addDataIndexSizeIntoMetaEntry(loadMetadataDetail,
-  loadMetadataDetail.getLoadName,
-  indexCarbonTable)
-}
-  })
-  SegmentStatusManager
-
.writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(tablePath),
-  loadMetadataDetails)
-} else {
-  throw new RuntimeException(
-"Not able to acquire the lock for table status updation for 
table " + databaseName +
-"." + indexCarbonTable.getTableName)
-}
-  } finally {
-if (statusLock != null) {
-  statusLock.unlock()
-}
-  }
-  // clear the indexSchema cache for the merged segments, as the index 
files and
-  // data files are rewritten after compaction
+  val segmentToLoadStartTimeMap: scala.collection.mutable.Map[String, 
java.lang.Long] =
+

[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534677505



##
File path: 
core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
##
@@ -241,13 +266,17 @@ public String 
writeMergeIndexFileBasedOnSegmentFile(String segmentId,
   break;
 }
   }
+  if (!table.isIndexTable()) {

Review comment:
   done, maintained mapping for SI and MT now.

##
File path: 
core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
##
@@ -277,21 +305,18 @@ public String 
writeMergeIndexFileBasedOnSegmentFile(String segmentId,
 LOGGER.error("unable to write segment file during merge index writing: 
" + ex.getMessage());
 throw ex;
   }
-  boolean status = SegmentFileStore.updateTableStatusFile(table, 
segmentId, newSegmentFileName,
-  table.getCarbonTableIdentifier().getTableId(), segmentFileStore);
-  if (!status) {
-// revert to original segment file as the table status update has 
failed.
-SegmentStatusManager.writeStringIntoFile(path, content);
-// delete merge index file.
-for (String file : mergeIndexFiles) {
-  FileFactory.getCarbonFile(file).delete();
-}
-// no need to delete index files, so return from here.
-return uuid;
-  }
 }
-for (CarbonFile file : indexFiles) {
-  file.delete();
+boolean status = SegmentFileStore.updateTableStatusFile(table, segmentId, 
newSegmentFileName,

Review comment:
   `mergeIndexBasedOnSegmentFile `is called when old store is used (here, 
already segment detail is loaded as success in tablestatus), it will modify 
size details and segment file name.





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




[GitHub] [carbondata] kunal642 commented on pull request #4010: [CARBONDATA-4050]Avoid redundant RPC calls to get file status when CarbonFile is instantiated with fileStatus constructor

2020-12-02 Thread GitBox


kunal642 commented on pull request #4010:
URL: https://github.com/apache/carbondata/pull/4010#issuecomment-737674089


   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




[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534677231



##
File path: 
core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
##
@@ -672,7 +678,7 @@ public static boolean isMaxQueryTimeoutExceeded(long 
fileTimestamp) {
 
 long minutesElapsed = (difference / (1000 * 60));
 
-return minutesElapsed > maxTime;
+return minutesElapsed >= maxTime;

Review comment:
   done

##
File path: core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
##
@@ -2558,8 +2558,11 @@ public static long getCarbonIndexSize(SegmentFileStore 
fileStore,
   // Get the total size of carbon data and the total size of carbon index
   public static HashMap getDataSizeAndIndexSize(String tablePath,
   Segment segment) throws IOException {
+SegmentFileStore fileStore = null;
 if (segment.getSegmentFileName() != null) {
-  SegmentFileStore fileStore = new SegmentFileStore(tablePath, 
segment.getSegmentFileName());
+  fileStore = new SegmentFileStore(tablePath, 
segment.getSegmentFileName());
+}
+if (segment.getSegmentFileName() != null && fileStore.getSegmentFile() != 
null) {

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




[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534677179



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -664,7 +723,8 @@ public static SegmentFile 
getSegmentFileForPhysicalDataPartitions(String tablePa
   CarbonFile[] listFiles = carbonFile.listFiles(new CarbonFileFilter() {
 @Override
 public boolean accept(CarbonFile file) {
-  return CarbonTablePath.isCarbonIndexFile(file.getAbsolutePath());
+  return file.getName().contains(uuid) && CarbonTablePath

Review comment:
   removed this part, not required.





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




[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534677017



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -439,6 +430,73 @@ public boolean accept(CarbonFile file) {
 return null;
   }
 
+  /**

Review comment:
   For SI, the flow is: in progress - merge index MT  - call load SI - 
merge index SI - writesegmentFile for SI - success SI  - writesegmentFile for 
MT - success MT. 
   Here, SI is reading index files from segment path as MT segment file is not 
yet written.





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




[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534676835



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -198,20 +208,17 @@ public boolean accept(CarbonFile file) {
 folderDetails.setRelative(isRelative);
 folderDetails.setPartitions(partitionNames);
 folderDetails.setStatus(SegmentStatus.SUCCESS.getMessage());
-for (CarbonFile file : carbonFiles) {
-  if (file.getName().endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT)) {
-folderDetails.setMergeFileName(file.getName());
-  } else {
-folderDetails.getFiles().add(file.getName());
-  }
-}
+setIndexFileNamesToFolderDetails(folderDetails, carbonFiles);
 segmentFile.addPath(location, folderDetails);
 String path = null;
 if (isMergeIndexFlow) {
   // in case of merge index flow, tasks are launched per partition and 
all the tasks
   // will be written to the same tmp folder, in that case taskNo is 
not unique.
   // To generate a unique fileName UUID is used
   path = writePath + "/" + CarbonUtil.generateUUID() + 
CarbonTablePath.SEGMENT_EXT;
+  if (readFileFooterFromCarbonDataFile) {
+path = writePath + "/" + timeStamp + CarbonTablePath.SEGMENT_EXT;

Review comment:
   removed unnecessary code from this method.





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




[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534676791



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -162,15 +162,25 @@ public static void writeSegmentFile(String tablePath, 
String segmentId, String t
* corresponding partitions.
*/
   public static void writeSegmentFile(String tablePath, final String taskNo, 
String location,
-  String timeStamp, List partitionNames, boolean isMergeIndexFlow) 
throws IOException {
-String tempFolderLoc = timeStamp + ".tmp";
-String writePath = CarbonTablePath.getSegmentFilesLocation(tablePath) + 
"/" + tempFolderLoc;
+  String timeStamp, List partitionNames, boolean isMergeIndexFlow,
+  boolean readFileFooterFromCarbonDataFile) throws IOException {
+String tempFolderLoc;
+String writePath;
+if (!readFileFooterFromCarbonDataFile) {
+  tempFolderLoc = timeStamp + ".tmp";
+  writePath = CarbonTablePath.getSegmentFilesLocation(tablePath) + "/" + 
tempFolderLoc;
+} else {
+  // If Alter merge index for old tables is triggered,
+  // directly write mergeindex file into segment file location
+  tempFolderLoc = location;
+  writePath = CarbonTablePath.getSegmentFilesLocation(tablePath);
+}
 CarbonFile carbonFile = FileFactory.getCarbonFile(writePath);
 if (!carbonFile.exists()) {
   carbonFile.mkdirs();
 }
 CarbonFile tempFolder;
-if (isMergeIndexFlow) {
+if (isMergeIndexFlow || readFileFooterFromCarbonDataFile) {

Review comment:
   removed unnecessary code from this method.





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




[GitHub] [carbondata] ShreelekhyaG commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


ShreelekhyaG commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534676638



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -162,15 +162,25 @@ public static void writeSegmentFile(String tablePath, 
String segmentId, String t
* corresponding partitions.
*/
   public static void writeSegmentFile(String tablePath, final String taskNo, 
String location,
-  String timeStamp, List partitionNames, boolean isMergeIndexFlow) 
throws IOException {
-String tempFolderLoc = timeStamp + ".tmp";
-String writePath = CarbonTablePath.getSegmentFilesLocation(tablePath) + 
"/" + tempFolderLoc;
+  String timeStamp, List partitionNames, boolean isMergeIndexFlow,
+  boolean readFileFooterFromCarbonDataFile) throws IOException {

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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4036: [WIP][Test CI]

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4036:
URL: https://github.com/apache/carbondata/pull/4036#issuecomment-737672816


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5035/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#issuecomment-737663951


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3277/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#issuecomment-737661221


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5034/
   



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




[GitHub] [carbondata] Zhangshunyu opened a new pull request #4036: [WIP][Test CI]

2020-12-02 Thread GitBox


Zhangshunyu opened a new pull request #4036:
URL: https://github.com/apache/carbondata/pull/4036


### 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




[GitHub] [carbondata] QiangCai commented on a change in pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


QiangCai commented on a change in pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#discussion_r534596239



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -1095,7 +1099,8 @@ public static void 
deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean
 // if execute command 'clean files' or the number of invisible 
segment info
 // exceeds the value of 'carbon.invisible.segments.preserve.count',
 // it need to append the invisible segment list to 
'tablestatus.history' file.
-if (isForceDeletion || (invisibleSegmentCnt > 
invisibleSegmentPreserveCnt)) {
+if (cleanStaleInprogress || cleanCompactedAndMFD || 
(invisibleSegmentCnt >

Review comment:
   do clean files move load metadata to history always





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




[GitHub] [carbondata] QiangCai commented on a change in pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


QiangCai commented on a change in pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#discussion_r534596239



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -1095,7 +1099,8 @@ public static void 
deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean
 // if execute command 'clean files' or the number of invisible 
segment info
 // exceeds the value of 'carbon.invisible.segments.preserve.count',
 // it need to append the invisible segment list to 
'tablestatus.history' file.
-if (isForceDeletion || (invisibleSegmentCnt > 
invisibleSegmentPreserveCnt)) {
+if (cleanStaleInprogress || cleanCompactedAndMFD || 
(invisibleSegmentCnt >

Review comment:
   does clean files move load metadata to history always





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




[GitHub] [carbondata] QiangCai commented on pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


QiangCai commented on pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#issuecomment-737611158


   @vikramahuja1001 please change title to describe what you change



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




[GitHub] [carbondata] QiangCai commented on a change in pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


QiangCai commented on a change in pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#discussion_r534596239



##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -1095,7 +1099,8 @@ public static void 
deleteLoadsAndUpdateMetadata(CarbonTable carbonTable, boolean
 // if execute command 'clean files' or the number of invisible 
segment info
 // exceeds the value of 'carbon.invisible.segments.preserve.count',
 // it need to append the invisible segment list to 
'tablestatus.history' file.
-if (isForceDeletion || (invisibleSegmentCnt > 
invisibleSegmentPreserveCnt)) {
+if (cleanStaleInprogress || cleanCompactedAndMFD || 
(invisibleSegmentCnt >

Review comment:
   no need move load metadata to history always

##
File path: 
core/src/main/java/org/apache/carbondata/core/statusmanager/SegmentStatusManager.java
##
@@ -1039,17 +1039,19 @@ private static void 
writeLoadMetadata(AbsoluteTableIdentifier identifier,
 }
   }
 
-  private static ReturnTuple isUpdateRequired(boolean isForceDeletion, 
CarbonTable carbonTable,
-  AbsoluteTableIdentifier absoluteTableIdentifier, LoadMetadataDetails[] 
details) {
+  private static ReturnTuple isUpdateRequired(boolean cleanStaleInProgress, 
boolean

Review comment:
   why remove isForceDeletion?

##
File path: 
core/src/main/java/org/apache/carbondata/core/util/DeleteLoadFolders.java
##
@@ -173,40 +176,62 @@ public boolean accept(CarbonFile file) {
   }
 
   private static boolean checkIfLoadCanBeDeleted(LoadMetadataDetails oneLoad,
-  boolean isForceDelete) {
-if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus() ||
-SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() ||
-SegmentStatus.INSERT_IN_PROGRESS == oneLoad.getSegmentStatus() ||
-SegmentStatus.INSERT_OVERWRITE_IN_PROGRESS == 
oneLoad.getSegmentStatus())
-&& oneLoad.getVisibility().equalsIgnoreCase("true")) {
-  if (isForceDelete) {
-return true;
-  }
-  long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
-  return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && 
CarbonUpdateUtil
-  .isMaxQueryTimeoutExceeded(deletionTime);
+  boolean cleanStaleInProgress, boolean cleanCompactedAndMFD) {
+if (oneLoad.getVisibility().equalsIgnoreCase("true")) {
+  return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, 
cleanStaleInProgress);
 }
-
 return false;
   }
 
   private static boolean checkIfLoadCanBeDeletedPhysically(LoadMetadataDetails 
oneLoad,
-  boolean isForceDelete) {
+  boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
 // Check if the segment is added externally and path is set then do not 
delete it
-if ((SegmentStatus.MARKED_FOR_DELETE == oneLoad.getSegmentStatus()
-|| SegmentStatus.COMPACTED == oneLoad.getSegmentStatus()) && 
(oneLoad.getPath() == null
-|| oneLoad.getPath().equalsIgnoreCase("NA"))) {
-  if (isForceDelete) {
-return true;
-  }
-  long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+if (oneLoad.getPath() == null || oneLoad.getPath().equalsIgnoreCase("NA")) 
{
+  return checkLoadDeletionLogic(oneLoad, cleanCompactedAndMFD, 
cleanStaleInProgress);
+}
+return false;
+  }
 
-  return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && 
CarbonUpdateUtil
+  private static Boolean checkLoadDeletionLogic(LoadMetadataDetails oneLoad,
+  boolean cleanCompactedAndMFD, boolean cleanStaleInProgress) {
+/*
+ * if cleanStaleInProgress == false and  cleanCompactedAndMFD == false, 
clean MFD and Compacted
+ *  segments after trashtimeout(7days) && query timeout(1 hr)
+ * if cleanStaleInProgress == false and  cleanCompactedAndMFD == true, 
clean MFD and Compacted
+ *  segments immediately
+ * if cleanStaleInProgress == true and  cleanCompactedAndMFD == false, 
clean Stale Inprogress
+ *  segments after 7 days(taking carbon.trash.retention.time value)
+ * if cleanStaleInProgress == true and  cleanCompactedAndMFD == true, 
clean MFD, Compacted and
+ *  stale inprogress segments immediately.
+ */
+if (!cleanCompactedAndMFD && !cleanStaleInProgress) {
+  if (SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || 
SegmentStatus
+  .MARKED_FOR_DELETE == oneLoad.getSegmentStatus()) {
+long deletionTime = oneLoad.getModificationOrDeletionTimestamp();
+return TrashUtil.isTrashRetentionTimeoutExceeded(deletionTime) && 
CarbonUpdateUtil
   .isMaxQueryTimeoutExceeded(deletionTime);
-
+  }
+  return false;
+} else if (cleanCompactedAndMFD && !cleanStaleInProgress) {
+  return SegmentStatus.COMPACTED == oneLoad.getSegmentStatus() || 
SegmentStatus
+.MARKED_FOR_DELETE == 

[GitHub] [carbondata] Zhangshunyu commented on pull request #4020: [CARBONDATA-4054] Support data size control for minor compaction

2020-12-02 Thread GitBox


Zhangshunyu commented on pull request #4020:
URL: https://github.com/apache/carbondata/pull/4020#issuecomment-737599642


   @ajantha-bhat @akashrn5 pls check



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4012: [CARBONDATA-4051] Geo spatial index algorithm improvement and UDFs enhancement

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4012:
URL: https://github.com/apache/carbondata/pull/4012#issuecomment-737358091


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3276/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4012: [CARBONDATA-4051] Geo spatial index algorithm improvement and UDFs enhancement

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4012:
URL: https://github.com/apache/carbondata/pull/4012#issuecomment-737354631


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5033/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4026: [CARBONDATA-4063] Refactor getBlockId and getShortBlockId functions

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4026:
URL: https://github.com/apache/carbondata/pull/4026#issuecomment-737338281


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3274/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4026: [CARBONDATA-4063] Refactor getBlockId and getShortBlockId functions

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4026:
URL: https://github.com/apache/carbondata/pull/4026#issuecomment-737334203


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5031/
   



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




[GitHub] [carbondata] shenjiayu17 commented on a change in pull request #4012: [CARBONDATA-4051] Geo spatial index algorithm improvement and UDFs enhancement

2020-12-02 Thread GitBox


shenjiayu17 commented on a change in pull request #4012:
URL: https://github.com/apache/carbondata/pull/4012#discussion_r534240585



##
File path: 
geo/src/main/java/org/apache/carbondata/geo/scan/expression/PolygonRangeListExpression.java
##
@@ -0,0 +1,211 @@
+/*
+ * 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.geo.scan.expression;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.scan.expression.ColumnExpression;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.ExpressionResult;
+import org.apache.carbondata.core.scan.expression.UnknownExpression;
+import 
org.apache.carbondata.core.scan.expression.conditional.ConditionalExpression;
+import org.apache.carbondata.core.scan.filter.executer.FilterExecutor;
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+import 
org.apache.carbondata.core.scan.filter.resolver.RowLevelFilterResolverImpl;
+import org.apache.carbondata.geo.GeoConstants;
+import org.apache.carbondata.geo.GeoHashUtils;
+import 
org.apache.carbondata.geo.scan.filter.executor.PolygonFilterExecutorImpl;
+
+/**
+ * InPolygonRangeList expression processor. It inputs the InPolygonRangeList 
string to
+ * the Geo implementation's query method, inputs lists of range of IDs and is 
to be calculated
+ * the and/or/diff range list to filter. And then, build InExpression with 
list of all the IDs
+ * present in those list of ranges.
+ */
+@InterfaceAudience.Internal
+public class PolygonRangeListExpression extends UnknownExpression
+implements ConditionalExpression {
+
+  private String polygonRangeList;
+
+  private String opType;
+
+  private List ranges = new ArrayList();
+
+  private ColumnExpression column;
+
+  private static final ExpressionResult trueExpRes =
+  new ExpressionResult(DataTypes.BOOLEAN, true);
+
+  private static final ExpressionResult falseExpRes =
+  new ExpressionResult(DataTypes.BOOLEAN, false);
+
+  public PolygonRangeListExpression(String polygonRangeList, String opType, 
String columnName) {
+this.polygonRangeList = polygonRangeList;
+this.opType = opType;
+this.column = new ColumnExpression(columnName, DataTypes.LONG);
+  }
+
+  private void processExpression() {
+try {
+  // 1. parse the range list string
+  List rangeLists = new ArrayList<>();
+  Pattern pattern = Pattern.compile(GeoConstants.RANGELIST_REG_EXPRESSION);
+  Matcher matcher = pattern.matcher(polygonRangeList);
+  while (matcher.find()) {
+String matchedStr = matcher.group();
+rangeLists.add(matchedStr);
+  }
+  // 2. process the range lists
+  if (rangeLists.size() > 0) {
+List processedRangeList = 
getRangeListFromString(rangeLists.get(0));
+for (int i = 1; i < rangeLists.size(); i++) {
+  List tempRangeList = 
getRangeListFromString(rangeLists.get(i));
+  processedRangeList = GeoHashUtils.processRangeList(
+processedRangeList, tempRangeList, opType);
+}
+ranges = processedRangeList;
+GeoHashUtils.validateRangeList(ranges);
+  }
+} catch (Exception e) {
+  throw new RuntimeException(e);
+}
+  }
+
+  private void sortRange(List rangeList) {
+rangeList.sort(new Comparator() {
+  @Override
+  public int compare(Long[] x, Long[] y) {
+return Long.compare(x[0], y[0]);
+  }
+});
+  }
+
+  private void combineRange(List rangeList) {
+if (rangeList.size() > 1) {
+  for (int i = 0, j = i + 1; i < rangeList.size() - 1; i++, j++) {
+long previousEnd = rangeList.get(i)[1];
+long nextStart = rangeList.get(j)[0];
+if (previousEnd 

[GitHub] [carbondata] shenjiayu17 commented on a change in pull request #4012: [CARBONDATA-4051] Geo spatial index algorithm improvement and UDFs enhancement

2020-12-02 Thread GitBox


shenjiayu17 commented on a change in pull request #4012:
URL: https://github.com/apache/carbondata/pull/4012#discussion_r534240300



##
File path: 
geo/src/main/java/org/apache/carbondata/geo/scan/expression/PolygonRangeListExpression.java
##
@@ -0,0 +1,211 @@
+/*
+ * 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.geo.scan.expression;
+
+import java.util.ArrayList;
+import java.util.Arrays;
+import java.util.Comparator;
+import java.util.List;
+import java.util.regex.Matcher;
+import java.util.regex.Pattern;
+
+import org.apache.carbondata.common.annotations.InterfaceAudience;
+import org.apache.carbondata.core.datastore.block.SegmentProperties;
+import org.apache.carbondata.core.metadata.datatype.DataTypes;
+import org.apache.carbondata.core.scan.expression.ColumnExpression;
+import org.apache.carbondata.core.scan.expression.Expression;
+import org.apache.carbondata.core.scan.expression.ExpressionResult;
+import org.apache.carbondata.core.scan.expression.UnknownExpression;
+import 
org.apache.carbondata.core.scan.expression.conditional.ConditionalExpression;
+import org.apache.carbondata.core.scan.filter.executer.FilterExecutor;
+import org.apache.carbondata.core.scan.filter.intf.ExpressionType;
+import org.apache.carbondata.core.scan.filter.intf.RowIntf;
+import org.apache.carbondata.core.scan.filter.resolver.FilterResolverIntf;
+import 
org.apache.carbondata.core.scan.filter.resolver.RowLevelFilterResolverImpl;
+import org.apache.carbondata.geo.GeoConstants;
+import org.apache.carbondata.geo.GeoHashUtils;
+import 
org.apache.carbondata.geo.scan.filter.executor.PolygonFilterExecutorImpl;
+
+/**
+ * InPolygonRangeList expression processor. It inputs the InPolygonRangeList 
string to
+ * the Geo implementation's query method, inputs lists of range of IDs and is 
to be calculated
+ * the and/or/diff range list to filter. And then, build InExpression with 
list of all the IDs
+ * present in those list of ranges.
+ */
+@InterfaceAudience.Internal
+public class PolygonRangeListExpression extends UnknownExpression
+implements ConditionalExpression {
+
+  private String polygonRangeList;
+
+  private String opType;
+
+  private List ranges = new ArrayList();
+
+  private ColumnExpression column;
+
+  private static final ExpressionResult trueExpRes =
+  new ExpressionResult(DataTypes.BOOLEAN, true);
+
+  private static final ExpressionResult falseExpRes =
+  new ExpressionResult(DataTypes.BOOLEAN, false);
+
+  public PolygonRangeListExpression(String polygonRangeList, String opType, 
String columnName) {
+this.polygonRangeList = polygonRangeList;
+this.opType = opType;
+this.column = new ColumnExpression(columnName, DataTypes.LONG);
+  }
+
+  private void processExpression() {
+try {
+  // 1. parse the range list string
+  List rangeLists = new ArrayList<>();
+  Pattern pattern = Pattern.compile(GeoConstants.RANGELIST_REG_EXPRESSION);
+  Matcher matcher = pattern.matcher(polygonRangeList);
+  while (matcher.find()) {
+String matchedStr = matcher.group();
+rangeLists.add(matchedStr);
+  }
+  // 2. process the range lists
+  if (rangeLists.size() > 0) {
+List processedRangeList = 
getRangeListFromString(rangeLists.get(0));
+for (int i = 1; i < rangeLists.size(); i++) {
+  List tempRangeList = 
getRangeListFromString(rangeLists.get(i));
+  processedRangeList = GeoHashUtils.processRangeList(
+processedRangeList, tempRangeList, opType);
+}
+ranges = processedRangeList;
+GeoHashUtils.validateRangeList(ranges);
+  }
+} catch (Exception e) {
+  throw new RuntimeException(e);
+}
+  }
+
+  private void sortRange(List rangeList) {
+rangeList.sort(new Comparator() {
+  @Override
+  public int compare(Long[] x, Long[] y) {
+return Long.compare(x[0], y[0]);
+  }
+});
+  }
+
+  private void combineRange(List rangeList) {
+if (rangeList.size() > 1) {
+  for (int i = 0, j = i + 1; i < rangeList.size() - 1; i++, j++) {
+long previousEnd = rangeList.get(i)[1];
+long nextStart = rangeList.get(j)[0];
+if (previousEnd 

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4010: [CARBONDATA-4050]Avoid redundant RPC calls to get file status when CarbonFile is instantiated with fileStatus constructor

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4010:
URL: https://github.com/apache/carbondata/pull/4010#issuecomment-737283454


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3272/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4002: [CARBONDATA-4046] Handled multiple partition columns for partition cache

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4002:
URL: https://github.com/apache/carbondata/pull/4002#issuecomment-737280752


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5029/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4002: [CARBONDATA-4046] Handled multiple partition columns for partition cache

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4002:
URL: https://github.com/apache/carbondata/pull/4002#issuecomment-737277740


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3273/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4010: [CARBONDATA-4050]Avoid redundant RPC calls to get file status when CarbonFile is instantiated with fileStatus constructor

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4010:
URL: https://github.com/apache/carbondata/pull/4010#issuecomment-737277105


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5028/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4017: [CARBONDATA-4022] Fix invalid path issue for segment added through alter table add segment query.

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4017:
URL: https://github.com/apache/carbondata/pull/4017#issuecomment-737277019


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3271/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4017: [CARBONDATA-4022] Fix invalid path issue for segment added through alter table add segment query.

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4017:
URL: https://github.com/apache/carbondata/pull/4017#issuecomment-737276707


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5027/
   



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




[GitHub] [carbondata] marchpure commented on a change in pull request #4026: [CARBONDATA-4063] Refactor getBlockId and getShortBlockId functions

2020-12-02 Thread GitBox


marchpure commented on a change in pull request #4026:
URL: https://github.com/apache/carbondata/pull/4026#discussion_r534211936



##
File path: 
core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
##
@@ -635,44 +604,89 @@ public static String getCarbonMergeIndexExtension() {
 return MERGE_INDEX_FILE_EXT;
   }
 
+  /**
+   * Generate the blockId as per the block path
+   *
+   * @param identifier
+   * @param filePath
+   * @param segmentId
+   * @param isPartitionTable
+   * @return blockid, which is the identify of a block
+   */
+  public static String getBlockId(AbsoluteTableIdentifier identifier, String 
filePath,
+  String segmentId, boolean isPartitionTable) {
+String blockName = filePath.substring(filePath.lastIndexOf(
+CarbonCommonConstants.FILE_SEPARATOR) + 1);
+String tablePath = identifier.getTablePath();
+
+String partitionDir = "";
+// 1. For block of Added Segments, The BlockId consistsof
+//
+if (!filePath.startsWith(tablePath)) {
+  partitionDir = getPartitionDir(tablePath, filePath, blockName);
+  return partitionDir.replace(CarbonCommonConstants.FILE_SEPARATOR, "#")
+  + CarbonCommonConstants.FILE_SEPARATOR + segmentId
+  + CarbonCommonConstants.FILE_SEPARATOR + blockName;
+}
+
+// 2. For block of partitiontable, The BlockId consistsof
+//
+if (isPartitionTable) {
+  partitionDir = getPartitionDir(tablePath, filePath, blockName);
+  return partitionDir.replace(CarbonCommonConstants.FILE_SEPARATOR, "#")
+  + CarbonCommonConstants.FILE_SEPARATOR + blockName;
+}
+
+// 3. For nonpartitiontable, The BlockId consistsof
+//
+return segmentId + CarbonCommonConstants.FILE_SEPARATOR + blockName;
+  }
+
   /**
* This method will remove strings in path and return short block id
*
* @param blockId
* @return shortBlockId
*/
   public static String getShortBlockId(String blockId) {
-String blockIdWithCompressorName =
-blockId.replace(PARTITION_PREFIX + "0" + 
CarbonCommonConstants.FILE_SEPARATOR, "")
-.replace(SEGMENT_PREFIX, "").replace(BATCH_PREFIX, 
CarbonCommonConstants.UNDERSCORE)
-.replace(DATA_PART_PREFIX, "").replace(CARBON_DATA_EXT, "");
-// to remove compressor name
-if (!blockId.equalsIgnoreCase(blockIdWithCompressorName)) {
-  int index = blockIdWithCompressorName.lastIndexOf(POINT);
-  int fileSeperatorIndex = 
blockIdWithCompressorName.lastIndexOf(File.separator);
-  if (index != -1) {
-String modifiedBlockId;
-if (index > fileSeperatorIndex) {
-  // Default case when path ends with compressor name.
-  // Example: 0/0-0_0-0-0-1600789595862.snappy
-  modifiedBlockId =
-  
blockIdWithCompressorName.replace(blockIdWithCompressorName.substring(index), 
"");
-} else {
-  // in case of CACHE_LEVEL = BLOCKLET, blockId path contains both 
block id and blocklet id
-  // so check for next file seperator and remove compressor name.
-  // Example: 0/0-0_0-0-0-1600789595862.snappy/0
-  modifiedBlockId = blockIdWithCompressorName
-  .replace(blockIdWithCompressorName.substring(index, 
fileSeperatorIndex), "");
-}
-return modifiedBlockId;
-  } else {
-return blockIdWithCompressorName;
-  }
-} else {
-  return blockIdWithCompressorName;
+// 1. If the blockid is already shortblockid, return directly
+int suffixIndex = blockId.lastIndexOf(CARBON_DATA_EXT);
+if (suffixIndex < 0) {
+  return blockId;
 }
+// 2. get the filepath. in the type of 
part=a/part-0-0_batchno0-0-0-1597409791503.snappy
+String filePath = blockId.substring(0, suffixIndex);
+
+// 3. get the compressor name, in the type of '.snappy'/'.zstd'/'.gzip'
+String compressorName = filePath.substring(filePath.lastIndexOf(POINT));
+
+// 4. get rid of 'Part0/' 'Segment_' '_batchno' 'part-' '.carbondata' and 
compressorname
+return blockId.replace(compressorName, "")
+.replace(PARTITION_PREFIX + "0" + 
CarbonCommonConstants.FILE_SEPARATOR, "")
+.replace(SEGMENT_PREFIX, "")
+.replace(BATCH_PREFIX, CarbonCommonConstants.UNDERSCORE)
+.replace(DATA_PART_PREFIX, "")
+.replace(CARBON_DATA_EXT, "");
+  }
+
+  /**
+   * get the partition path in the block path
+   *
+   * @param tablePath
+   * @param filePath
+   * @param blockName
+   * @return blockid, which is the identify of a block
+   */
+  public static String getPartitionDir(String tablePath, String filePath, 
String blockName) {
+// The filepath is consist with 
+// The partitionPath is the string truncated between tablePath and 
blockName
+if (!filePath.startsWith(tablePath)) {
+  return filePath.substring(0, filePath.length() - blockName.length());
+}
+return filePath.substring(tablePath.length() + 1, 

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#issuecomment-737256672


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3270/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#issuecomment-737253057


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5026/
   



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




[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


vikramahuja1001 commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534175368



##
File path: 
integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/util/SecondaryIndexUtil.scala
##
@@ -226,85 +233,59 @@ object SecondaryIndexUtil {
   validSegmentsToUse.toList.asJava,
   indexCarbonTable)
   }
-  mergedSegments.asScala.map { seg =>
-val file = SegmentFileStore.writeSegmentFile(
-  indexCarbonTable,
-  seg.getLoadName,
-  carbonLoadModel.getFactTimeStamp.toString,
-  null,
-  null)
-val segment = new Segment(seg.getLoadName, file)
-SegmentFileStore.updateTableStatusFile(indexCarbonTable,
-  seg.getLoadName,
-  file,
-  indexCarbonTable.getCarbonTableIdentifier.getTableId,
-  new SegmentFileStore(tablePath, segment.getSegmentFileName))
-segment
-  }
-
-  val statusLock =
-new 
SegmentStatusManager(indexCarbonTable.getAbsoluteTableIdentifier).getTableStatusLock
-  try {
-val retryCount = 
CarbonLockUtil.getLockProperty(CarbonCommonConstants
-  .NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK,
-  
CarbonCommonConstants.NUMBER_OF_TRIES_FOR_CONCURRENT_LOCK_DEFAULT)
-val maxTimeout = 
CarbonLockUtil.getLockProperty(CarbonCommonConstants
-  .MAX_TIMEOUT_FOR_CONCURRENT_LOCK,
-  CarbonCommonConstants.MAX_TIMEOUT_FOR_CONCURRENT_LOCK_DEFAULT)
-if (statusLock.lockWithRetries(retryCount, maxTimeout)) {
-  val endTime = System.currentTimeMillis()
-  val loadMetadataDetails = SegmentStatusManager
-.readLoadMetadata(indexCarbonTable.getMetadataPath)
-  loadMetadataDetails.foreach(loadMetadataDetail => {
-if (rebuiltSegments.contains(loadMetadataDetail.getLoadName)) {
-  
loadMetadataDetail.setLoadStartTime(carbonLoadModel.getFactTimeStamp)
-  loadMetadataDetail.setLoadEndTime(endTime)
-  CarbonLoaderUtil
-.addDataIndexSizeIntoMetaEntry(loadMetadataDetail,
-  loadMetadataDetail.getLoadName,
-  indexCarbonTable)
-}
-  })
-  SegmentStatusManager
-
.writeLoadDetailsIntoFile(CarbonTablePath.getTableStatusFilePath(tablePath),
-  loadMetadataDetails)
-} else {
-  throw new RuntimeException(
-"Not able to acquire the lock for table status updation for 
table " + databaseName +
-"." + indexCarbonTable.getTableName)
-}
-  } finally {
-if (statusLock != null) {
-  statusLock.unlock()
-}
-  }
-  // clear the indexSchema cache for the merged segments, as the index 
files and
-  // data files are rewritten after compaction
+  val segmentToLoadStartTimeMap: scala.collection.mutable.Map[String, 
java.lang.Long] =
+scala.collection.mutable.Map()
   if (mergedSegments.size > 0) {

Review comment:
   this if is not required





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




[GitHub] [carbondata] akashrn5 commented on a change in pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


akashrn5 commented on a change in pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#discussion_r534107457



##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -162,15 +162,25 @@ public static void writeSegmentFile(String tablePath, 
String segmentId, String t
* corresponding partitions.
*/
   public static void writeSegmentFile(String tablePath, final String taskNo, 
String location,
-  String timeStamp, List partitionNames, boolean isMergeIndexFlow) 
throws IOException {
-String tempFolderLoc = timeStamp + ".tmp";
-String writePath = CarbonTablePath.getSegmentFilesLocation(tablePath) + 
"/" + tempFolderLoc;
+  String timeStamp, List partitionNames, boolean isMergeIndexFlow,
+  boolean readFileFooterFromCarbonDataFile) throws IOException {

Review comment:
   rename to some meaningful , its confusing now 
`readFileFooterFromCarbonDataFile`

##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -198,20 +208,17 @@ public boolean accept(CarbonFile file) {
 folderDetails.setRelative(isRelative);
 folderDetails.setPartitions(partitionNames);
 folderDetails.setStatus(SegmentStatus.SUCCESS.getMessage());
-for (CarbonFile file : carbonFiles) {
-  if (file.getName().endsWith(CarbonTablePath.MERGE_INDEX_FILE_EXT)) {
-folderDetails.setMergeFileName(file.getName());
-  } else {
-folderDetails.getFiles().add(file.getName());
-  }
-}
+setIndexFileNamesToFolderDetails(folderDetails, carbonFiles);
 segmentFile.addPath(location, folderDetails);
 String path = null;
 if (isMergeIndexFlow) {
   // in case of merge index flow, tasks are launched per partition and 
all the tasks
   // will be written to the same tmp folder, in that case taskNo is 
not unique.
   // To generate a unique fileName UUID is used
   path = writePath + "/" + CarbonUtil.generateUUID() + 
CarbonTablePath.SEGMENT_EXT;
+  if (readFileFooterFromCarbonDataFile) {
+path = writePath + "/" + timeStamp + CarbonTablePath.SEGMENT_EXT;

Review comment:
   can you please check why this change is added, if valid change, please 
add a proper comment explaining the scenario why this required

##
File path: 
core/src/main/java/org/apache/carbondata/core/metadata/SegmentFileStore.java
##
@@ -439,6 +430,73 @@ public boolean accept(CarbonFile file) {
 return null;
   }
 
+  /**

Review comment:
   please check if the flow is in below order in case of SI
   
   in progress - merge index MT - update or generate segment file for MT - call 
load SI - (all SI functions) - success MT

##
File path: 
core/src/main/java/org/apache/carbondata/core/writer/CarbonIndexFileMergeWriter.java
##
@@ -140,33 +141,36 @@ private String mergeCarbonIndexFilesOfSegment(String 
segmentId,
 groupIndexesBySegment(fileStore.getCarbonIndexMapWithFullPath());
 SegmentFileStore.FolderDetails folderDetails = null;
 for (Map.Entry> entry : 
indexLocationMap.entrySet()) {
-  String mergeIndexFile =
-  writeMergeIndexFile(null, partitionPath, entry.getValue(), 
segmentId, uuid);
-  folderDetails = new SegmentFileStore.FolderDetails();
-  folderDetails.setMergeFileName(mergeIndexFile);
-  folderDetails.setStatus("Success");
-  if (partitionPath.startsWith(tablePath)) {
-partitionPath = partitionPath.substring(tablePath.length() + 1);
-List partitions = new 
ArrayList<>(Arrays.asList(partitionPath.split("/")));
+  Map> mergeToIndexFileMap = 
fileStore.getCarbonMergeFileToIndexFilesMap();

Review comment:
   please cross verify, i think this changes not required if you handle in 
compact merge index for new tables whether to go for merging index files or no 
for new tables based on segment file and list files in the absence of segment 
file.

##
File path: 
core/src/main/java/org/apache/carbondata/core/mutate/CarbonUpdateUtil.java
##
@@ -672,7 +678,7 @@ public static boolean isMaxQueryTimeoutExceeded(long 
fileTimestamp) {
 
 long minutesElapsed = (difference / (1000 * 60));
 
-return minutesElapsed > maxTime;
+return minutesElapsed >= maxTime;

Review comment:
   please revert this

##
File path: core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
##
@@ -2558,8 +2558,11 @@ public static long getCarbonIndexSize(SegmentFileStore 
fileStore,
   // Get the total size of carbon data and the total size of carbon index
   public static HashMap getDataSizeAndIndexSize(String tablePath,
   Segment segment) throws IOException {
+SegmentFileStore fileStore = null;
 if (segment.getSegmentFileName() != null) {
-  SegmentFileStore fileStore = new SegmentFileStore(tablePath, 

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#issuecomment-737218591


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3269/
   



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




[GitHub] [carbondata] kunal642 commented on pull request #4017: [CARBONDATA-4022] Fix invalid path issue for segment added through alter table add segment query.

2020-12-02 Thread GitBox


kunal642 commented on pull request #4017:
URL: https://github.com/apache/carbondata/pull/4017#issuecomment-737215466


   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




[GitHub] [carbondata] kunal642 commented on pull request #4002: [CARBONDATA-4046] Handled multiple partition columns for partition cache

2020-12-02 Thread GitBox


kunal642 commented on pull request #4002:
URL: https://github.com/apache/carbondata/pull/4002#issuecomment-737215552


   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




[GitHub] [carbondata] kunal642 commented on pull request #4010: [CARBONDATA-4050]Avoid redundant RPC calls to get file status when CarbonFile is instantiated with fileStatus constructor

2020-12-02 Thread GitBox


kunal642 commented on pull request #4010:
URL: https://github.com/apache/carbondata/pull/4010#issuecomment-737215345


   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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#issuecomment-737213153


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5025/
   



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




[GitHub] [carbondata] shenjiayu17 commented on a change in pull request #4012: [CARBONDATA-4051] Geo spatial index algorithm improvement and UDFs enhancement

2020-12-02 Thread GitBox


shenjiayu17 commented on a change in pull request #4012:
URL: https://github.com/apache/carbondata/pull/4012#discussion_r534107071



##
File path: geo/src/main/java/org/apache/carbondata/geo/GeoHashUtils.java
##
@@ -0,0 +1,411 @@
+/*
+ * 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.geo;
+
+import java.math.BigDecimal;
+import java.util.ArrayList;
+import java.util.Collections;
+import java.util.Comparator;
+import java.util.List;
+import java.util.Objects;
+
+public class GeoHashUtils {
+
+  /**
+   * Get the degree of each grid in the east-west direction.
+   *
+   * @param originLatitude the origin point latitude
+   * @param gridSize the grid size
+   * @return Delta X is the degree of each grid in the east-west direction
+   */
+  public static double getDeltaX(double originLatitude, int gridSize) {
+double mCos = Math.cos(originLatitude * Math.PI / 
GeoConstants.CONVERT_FACTOR);
+return (GeoConstants.CONVERT_FACTOR * gridSize) / (Math.PI * 
GeoConstants.EARTH_RADIUS * mCos);
+  }
+
+  /**
+   * Get the degree of each grid in the north-south direction.
+   *
+   * @param gridSize the grid size
+   * @return Delta Y is the degree of each grid in the north-south direction
+   */
+  public static double getDeltaY(int gridSize) {
+return (GeoConstants.CONVERT_FACTOR * gridSize) / (Math.PI * 
GeoConstants.EARTH_RADIUS);
+  }
+
+  /**
+   * Calculate the number of knives cut
+   *
+   * @param gridSize the grid size
+   * @param originLatitude the origin point latitude
+   * @return The number of knives cut
+   */
+  public static int getCutCount(int gridSize, double originLatitude) {
+double deltaX = getDeltaX(originLatitude, gridSize);
+int countX = Double.valueOf(
+Math.ceil(Math.log(2 * GeoConstants.CONVERT_FACTOR / deltaX) / 
Math.log(2))).intValue();
+double deltaY = getDeltaY(gridSize);
+int countY = Double.valueOf(
+Math.ceil(Math.log(GeoConstants.CONVERT_FACTOR / deltaY) / 
Math.log(2))).intValue();
+return Math.max(countX, countY);
+  }
+
+  /**
+   * Convert input longitude and latitude to GeoID
+   *
+   * @param longitude Longitude, the actual longitude and latitude are 
processed by * coefficient,
+   *  and the floating-point calculation is converted to 
integer calculation
+   * @param latitude Latitude, the actual longitude and latitude are processed 
by * coefficient,
+   *  and the floating-point calculation is converted to 
integer calculation.
+   * @param oriLatitude the origin point latitude
+   * @param gridSize the grid size
+   * @return GeoID
+   */
+  public static long lonLat2GeoID(long longitude, long latitude, double 
oriLatitude, int gridSize) {
+long longtitudeByRatio = longitude * 
GeoConstants.CONVERSION_FACTOR_FOR_ACCURACY;
+long latitudeByRatio = latitude * 
GeoConstants.CONVERSION_FACTOR_FOR_ACCURACY;
+int[] ij = lonLat2ColRow(longtitudeByRatio, latitudeByRatio, oriLatitude, 
gridSize);
+return colRow2GeoID(ij[0], ij[1]);
+  }
+
+  /**
+   * Calculate geo id through grid index coordinates, the row and column of 
grid coordinates
+   * can be transformed by latitude and longitude
+   *
+   * @param longitude Longitude, the actual longitude and latitude are 
processed by * coefficient,
+   * and the floating-point calculation is converted to integer calculation
+   * @param latitude Latitude, the actual longitude and latitude are processed 
by * coefficient,
+   * and the floating-point calculation is converted to integer calculation
+   * @param oriLatitude the latitude of origin point,which is used to 
calculate the deltaX and cut
+   * level.
+   * @param gridSize the size of minimal grid after cut
+   * @return Grid ID value [row, column], column starts from 1
+   */
+  public static int[] lonLat2ColRow(long longitude, long latitude, double 
oriLatitude,
+  int gridSize) {
+int cutLevel = getCutCount(gridSize, oriLatitude);
+int column = (int) Math.floor(longitude / getDeltaX(oriLatitude, gridSize) 
/
+GeoConstants.CONVERSION_RATIO) + (1 << (cutLevel - 1));
+int row = (int) Math.floor(latitude / getDeltaY(gridSize) /
+

[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#issuecomment-737177718


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3267/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4035: [CARBONDATA-4067]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#issuecomment-737176415


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5023/
   



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




[GitHub] [carbondata] QiangCai commented on a change in pull request #4026: [CARBONDATA-4063] Refactor getBlockId and getShortBlockId functions

2020-12-02 Thread GitBox


QiangCai commented on a change in pull request #4026:
URL: https://github.com/apache/carbondata/pull/4026#discussion_r534097978



##
File path: 
core/src/main/java/org/apache/carbondata/core/util/path/CarbonTablePath.java
##
@@ -635,44 +604,89 @@ public static String getCarbonMergeIndexExtension() {
 return MERGE_INDEX_FILE_EXT;
   }
 
+  /**
+   * Generate the blockId as per the block path
+   *
+   * @param identifier
+   * @param filePath
+   * @param segmentId
+   * @param isPartitionTable
+   * @return blockid, which is the identify of a block
+   */
+  public static String getBlockId(AbsoluteTableIdentifier identifier, String 
filePath,
+  String segmentId, boolean isPartitionTable) {
+String blockName = filePath.substring(filePath.lastIndexOf(
+CarbonCommonConstants.FILE_SEPARATOR) + 1);
+String tablePath = identifier.getTablePath();
+
+String partitionDir = "";
+// 1. For block of Added Segments, The BlockId consistsof
+//
+if (!filePath.startsWith(tablePath)) {
+  partitionDir = getPartitionDir(tablePath, filePath, blockName);
+  return partitionDir.replace(CarbonCommonConstants.FILE_SEPARATOR, "#")
+  + CarbonCommonConstants.FILE_SEPARATOR + segmentId
+  + CarbonCommonConstants.FILE_SEPARATOR + blockName;
+}
+
+// 2. For block of partitiontable, The BlockId consistsof
+//
+if (isPartitionTable) {
+  partitionDir = getPartitionDir(tablePath, filePath, blockName);
+  return partitionDir.replace(CarbonCommonConstants.FILE_SEPARATOR, "#")
+  + CarbonCommonConstants.FILE_SEPARATOR + blockName;
+}
+
+// 3. For nonpartitiontable, The BlockId consistsof
+//
+return segmentId + CarbonCommonConstants.FILE_SEPARATOR + blockName;
+  }
+
   /**
* This method will remove strings in path and return short block id
*
* @param blockId
* @return shortBlockId
*/
   public static String getShortBlockId(String blockId) {
-String blockIdWithCompressorName =
-blockId.replace(PARTITION_PREFIX + "0" + 
CarbonCommonConstants.FILE_SEPARATOR, "")
-.replace(SEGMENT_PREFIX, "").replace(BATCH_PREFIX, 
CarbonCommonConstants.UNDERSCORE)
-.replace(DATA_PART_PREFIX, "").replace(CARBON_DATA_EXT, "");
-// to remove compressor name
-if (!blockId.equalsIgnoreCase(blockIdWithCompressorName)) {
-  int index = blockIdWithCompressorName.lastIndexOf(POINT);
-  int fileSeperatorIndex = 
blockIdWithCompressorName.lastIndexOf(File.separator);
-  if (index != -1) {
-String modifiedBlockId;
-if (index > fileSeperatorIndex) {
-  // Default case when path ends with compressor name.
-  // Example: 0/0-0_0-0-0-1600789595862.snappy
-  modifiedBlockId =
-  
blockIdWithCompressorName.replace(blockIdWithCompressorName.substring(index), 
"");
-} else {
-  // in case of CACHE_LEVEL = BLOCKLET, blockId path contains both 
block id and blocklet id
-  // so check for next file seperator and remove compressor name.
-  // Example: 0/0-0_0-0-0-1600789595862.snappy/0
-  modifiedBlockId = blockIdWithCompressorName
-  .replace(blockIdWithCompressorName.substring(index, 
fileSeperatorIndex), "");
-}
-return modifiedBlockId;
-  } else {
-return blockIdWithCompressorName;
-  }
-} else {
-  return blockIdWithCompressorName;
+// 1. If the blockid is already shortblockid, return directly
+int suffixIndex = blockId.lastIndexOf(CARBON_DATA_EXT);
+if (suffixIndex < 0) {
+  return blockId;
 }
+// 2. get the filepath. in the type of 
part=a/part-0-0_batchno0-0-0-1597409791503.snappy
+String filePath = blockId.substring(0, suffixIndex);
+
+// 3. get the compressor name, in the type of '.snappy'/'.zstd'/'.gzip'
+String compressorName = filePath.substring(filePath.lastIndexOf(POINT));
+
+// 4. get rid of 'Part0/' 'Segment_' '_batchno' 'part-' '.carbondata' and 
compressorname
+return blockId.replace(compressorName, "")
+.replace(PARTITION_PREFIX + "0" + 
CarbonCommonConstants.FILE_SEPARATOR, "")
+.replace(SEGMENT_PREFIX, "")
+.replace(BATCH_PREFIX, CarbonCommonConstants.UNDERSCORE)
+.replace(DATA_PART_PREFIX, "")
+.replace(CARBON_DATA_EXT, "");
+  }
+
+  /**
+   * get the partition path in the block path
+   *
+   * @param tablePath
+   * @param filePath
+   * @param blockName
+   * @return blockid, which is the identify of a block
+   */
+  public static String getPartitionDir(String tablePath, String filePath, 
String blockName) {
+// The filepath is consist with 
+// The partitionPath is the string truncated between tablePath and 
blockName
+if (!filePath.startsWith(tablePath)) {
+  return filePath.substring(0, filePath.length() - blockName.length());
+}
+return filePath.substring(tablePath.length() + 1, 

[GitHub] [carbondata] QiangCai commented on pull request #4013: [CARBONDATA-4062] Make clean files as data trash manager

2020-12-02 Thread GitBox


QiangCai commented on pull request #4013:
URL: https://github.com/apache/carbondata/pull/4013#issuecomment-737160943


   @akashrn5 @ajantha-bhat @vikramahuja1001 please review this PR



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-737155601


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3265/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #3988: [CARBONDATA-4037] Improve the table status and segment file writing

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #3988:
URL: https://github.com/apache/carbondata/pull/3988#issuecomment-737152317


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5021/
   



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




[jira] [Created] (CARBONDATA-4067) Change clean files behaviour to support cleaning of in progress segments

2020-12-02 Thread Vikram Ahuja (Jira)
Vikram Ahuja created CARBONDATA-4067:


 Summary: Change clean files behaviour to support cleaning of in 
progress segments
 Key: CARBONDATA-4067
 URL: https://issues.apache.org/jira/browse/CARBONDATA-4067
 Project: CarbonData
  Issue Type: Improvement
Reporter: Vikram Ahuja


Change clean files behaviour to support cleaning of in progress segments



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


[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4031: Presto UT optimization

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4031:
URL: https://github.com/apache/carbondata/pull/4031#issuecomment-737140156


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3263/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4031: Presto UT optimization

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4031:
URL: https://github.com/apache/carbondata/pull/4031#issuecomment-737138681


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5019/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4018: [CARBONDATA-4055]Fix creation of empty segment directory and meta entry when there is no update/insert data

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4018:
URL: https://github.com/apache/carbondata/pull/4018#issuecomment-737124721


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3262/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4018: [CARBONDATA-4055]Fix creation of empty segment directory and meta entry when there is no update/insert data

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4018:
URL: https://github.com/apache/carbondata/pull/4018#issuecomment-737120401


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5018/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4034: [WIP] support prestosql 333

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4034:
URL: https://github.com/apache/carbondata/pull/4034#issuecomment-737110915


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3261/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4035: [WIP]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#issuecomment-737107889


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5022/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4035: [WIP]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035#issuecomment-737107284


   Build Failed  with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3266/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4034: [WIP] support prestosql 333

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4034:
URL: https://github.com/apache/carbondata/pull/4034#issuecomment-737104390


   Build Failed  with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5017/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4020: [CARBONDATA-4054] Support data size control for minor compaction

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4020:
URL: https://github.com/apache/carbondata/pull/4020#issuecomment-737103815


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5016/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4020: [CARBONDATA-4054] Support data size control for minor compaction

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4020:
URL: https://github.com/apache/carbondata/pull/4020#issuecomment-737100516


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3260/
   



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




[GitHub] [carbondata] vikramahuja1001 opened a new pull request #4035: [WIP]: CleanFiles Behaviour Change

2020-12-02 Thread GitBox


vikramahuja1001 opened a new pull request #4035:
URL: https://github.com/apache/carbondata/pull/4035


### Why is this PR needed?
Change the behaviour change for clean files operation
   Old behaviour: Clean files command is by default force option and ignores 
query timeout.

### What changes were proposed in this PR?
   New Behaviour:
   
   default clean files behaviour(clean files on table t1): clean MFD and 
Compacted segments after query timeout(1 hr)
   clean files on table t1 options('force'='true'): clean MFD and Compacted 
segments immediately
   clean files on table t1 options('clean_inprgress'='true') : clean stale 
inprogress segments after 7 days(default behaviour)
   clean files on table t1 options('clean_inprgress'='true', 'force'='true') : 
clean MFD, Compacted and stale inprogress segments immediately.
   
### Does this PR introduce any user interface change?
- Yes. (please explain the change and update document)
   
### Is any new testcase added?
- No (previous test cases changed)
   
   
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4012: [CARBONDATA-4051] Geo spatial index algorithm improvement and UDFs enhancement

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4012:
URL: https://github.com/apache/carbondata/pull/4012#issuecomment-737092372


   Build Success with Spark 2.4.5, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbon_PR_Builder_2.4.5/3259/
   



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




[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4012: [CARBONDATA-4051] Geo spatial index algorithm improvement and UDFs enhancement

2020-12-02 Thread GitBox


CarbonDataQA2 commented on pull request #4012:
URL: https://github.com/apache/carbondata/pull/4012#issuecomment-737090938


   Build Success with Spark 2.3.4, Please check CI 
http://121.244.95.60:12545/job/ApacheCarbonPRBuilder2.3/5015/
   



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




  1   2   >