You are viewing a plain text version of this content. The canonical link for it is here.
Posted to dev@carbondata.apache.org by GitBox <gi...@apache.org> on 2021/05/25 17:08:59 UTC

[GitHub] [carbondata] vikramahuja1001 opened a new pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

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


    ### Why is this PR needed?
    To integrate Carbondata with Spark3.1.1
    
    ### What changes were proposed in this PR?
   Refactored code to add changes to support Spark 3.1.1 along with Spark 2.3 and 2.4 versions
       
    ### Does this PR introduce any user interface change?
    - No
   
    ### Is any new testcase added?
    - 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:
users@infra.apache.org



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644563696



##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.spark.sql.hive.execution.command
+
+import org.apache.spark.sql.{CarbonEnv, CarbonToSparkAdapter, Row, SparkSession}

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



[GitHub] [carbondata] Jeromestein commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
Jeromestein commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-888737358


   Hi guys, I wanna make a contribution to our community by working on the  Spark 3.1.1 support in CarbonData. First of all, I would like to build a connection with you. I've sent several emails to @vikramahuja1001, not response yet. How can I communicate with you guys? Maybe a slack link?


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

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644557814



##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -185,8 +224,390 @@ object CarbonToSparkAdapter {
   def getHiveExternalCatalog(sparkSession: SparkSession): HiveExternalCatalog = {
     sparkSession.sessionState.catalog.externalCatalog.asInstanceOf[HiveExternalCatalog]
   }
+
+  def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]) = {
+    FilePartition(index, files.toArray.toSeq)
+  }
+
+  def stringToTimestamp(timestamp: String): Option[Long] = {
+    DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp))
+  }
+
+  def getTableIdentifier(u: UnresolvedRelation): Some[TableIdentifier] = {
+    Some(u.tableIdentifier)
+  }
+
+  def dateToString(date: Int): String = {
+    DateTimeUtils.dateToString(date.toString.toInt)
+  }
+
+  def timeStampToString(timeStamp: Long): String = {
+    DateTimeUtils.timestampToString(timeStamp)
+  }

Review comment:
       done, added to common2.3and2.4




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-864899792


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644573460



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkSqlAdapter.scala
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.spark.sql
+
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.{Attribute, EmptyRow, Expression}
+import org.apache.spark.sql.execution.FileSourceScanExec
+import org.apache.spark.sql.execution.datasources.HadoopFsRelation
+import org.apache.spark.sql.types.StructType
+
+object SparkSqlAdapter {
+
+  def initSparkSQL(): Unit = {
+  }
+
+  def getScanForSegments(
+      @transient relation: HadoopFsRelation,

Review comment:
        FileSourceScanExec API is different in spark2.3, 2.4 and 3.1. Different number of arguments in all 3.




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655894035



##########
File path: mv/plan/src/main/spark2.3/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala
##########
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.mv.plans.modular
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, ExprId, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, ExprId, NamedExpression}

Review comment:
       Checkstyle was not running on this class initially, if changed back then checkstyle will fail.




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r645296443



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
##########
@@ -321,7 +321,7 @@ object DeleteExecution {
           deleteStatus = SegmentStatus.SUCCESS
         } catch {
           case e : MultipleMatchingException =>
-            LOGGER.error(e.getMessage)
+          LOGGER.error(e.getMessage)

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644544877



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/jobs/SparkBlockletIndexLoaderJob.scala
##########
@@ -160,9 +160,7 @@ class IndexLoaderRDD(
     val reader = indexFormat.createRecordReader(inputSplit, attemptContext)
     val iter = new Iterator[(TableBlockIndexUniqueIdentifier, BlockletIndexDetailsWithSchema)] {
       // in case of success, failure or cancellation clear memory and stop execution
-      context.addTaskCompletionListener { _ =>
-        reader.close()
-      }
+      CarbonToSparkAdapter.addTaskCompletionListener(reader.close())

Review comment:
       addTaskCompletionListener API changed in Spark 3.1




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

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



[GitHub] [carbondata] akashrn5 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r639004523



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -439,10 +445,31 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
     }
   }
 
+  private static long createTimeInstant(String dimensionValue, String dateFormat) {
+    String updatedDim = dimensionValue;
+    if (!dimensionValue.trim().contains(" ")) {
+      updatedDim += " 00:00:00";
+    }
+    Instant instant = Instant.from(ZonedDateTime
+        .of(LocalDateTime.parse(updatedDim, DateTimeFormatter.ofPattern(dateFormat)),
+        ZoneId.systemDefault()));
+    validateTimeStampRange(instant.getEpochSecond());
+    long us = Math.multiplyExact(instant.getEpochSecond(), 1000L);
+    return Math.addExact(us, instant.getNano() * 1000L);
+  }
+
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
     DateFormat dateFormatter = null;
     long timeValue;
+    if (Boolean.parseBoolean(CarbonProperties.getInstance().getProperty(CarbonCommonConstants
+        .CARBON_SPARK_VERSION_SPARK3))) {
+      try {
+        return createTimeInstant(dimensionValue, dateFormat);

Review comment:
       please add a comment here why the change in spark3 for any future reference

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -2644,5 +2644,4 @@ private CarbonCommonConstants() {
   public static final String CARBON_MAP_ORDER_PUSHDOWN = "carbon.mapOrderPushDown";
 
   public static final String CARBON_SDK_EMPTY_METADATA_PATH = "emptyMetadataFolder";
-

Review comment:
       revert this change

##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneIndexExample.scala
##########
@@ -61,37 +61,37 @@ object LuceneIndexExample {
       """.stripMargin)
 
     // 1. Compare the performance:
-
-    def time(code: => Unit): Double = {
-      val start = System.currentTimeMillis()
-      code
-      // return time in second
-      (System.currentTimeMillis() - start).toDouble / 1000
-    }
-
-    val timeWithoutLuceneIndex = time {
-
-      spark.sql(
-        s"""
-           | SELECT count(*)
-           | FROM personTable where id like '% test1 %'
-      """.stripMargin).show()
-
-    }
-
-    val timeWithLuceneIndex = time {
-
-      spark.sql(
-        s"""
-           | SELECT count(*)
-           | FROM personTable where TEXT_MATCH('id:test1')
-      """.stripMargin).show()
-
-    }
+    // TODO: Revert this after SPARK 3.1 fix

Review comment:
       can you please mention the spark JIRA id here, so that reviewer can know the exact problem

##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneIndexExample.scala
##########
@@ -30,7 +30,7 @@ object LuceneIndexExample {
 
   def main(args: Array[String]) {
     val spark = ExampleUtils.createSparkSession("LuceneIndexExample")
-    exampleBody(spark)
+      exampleBody(spark)

Review comment:
       revert this 

##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/StructuredStreamingExample.scala
##########
@@ -151,7 +151,7 @@ object StructuredStreamingExample {
           // Write data from socket stream to carbondata file
           qry = readSocketDF.writeStream
             .format("carbondata")
-            .trigger(ProcessingTime("5 seconds"))
+            .trigger(CarbonToSparkAdapter.getProcessingTime("5 seconds"))

Review comment:
       `getProcessingTime` calls the same API both in 2.4 and 3.1 and package also same for `Trigger` class, then why we need to put in different package, can you please explain?

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/CsvRDDHelper.scala
##########
@@ -26,7 +26,7 @@ import org.apache.hadoop.mapreduce.{TaskAttemptID, TaskType}
 import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.{CarbonToSparkAdapter, SparkSession}

Review comment:
       can we keep this class in the same package structure as old?

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
##########
@@ -360,9 +359,14 @@ object CarbonStore {
 
   private def validateTimeFormat(timestamp: String): Long = {
     try {
-      DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp)).get
+      CarbonToSparkAdapter.stringToTimestamp(timestamp) match {
+        case Some(value) => value
+        case _ =>
+          val errorMessage = "Error: Invalid load start time format: " + timestamp

Review comment:
       no need to define the same error message here, as its already handled in catch block, can just throw runtime exception

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
##########
@@ -67,6 +68,8 @@ object DDLStrategy extends SparkStrategy {
       case changeColumn: AlterTableChangeColumnCommand
         if isCarbonTable(changeColumn.tableName) =>
         ExecutedCommandExec(DDLHelper.changeColumn(changeColumn, sparkSession)) :: Nil
+      case changeColumn: CarbonAlterTableColRenameDataTypeChangeCommand =>

Review comment:
       command will always come as change column command. In which case this required as it wasn't there before

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/jobs/SparkBlockletIndexLoaderJob.scala
##########
@@ -160,9 +160,7 @@ class IndexLoaderRDD(
     val reader = indexFormat.createRecordReader(inputSplit, attemptContext)
     val iter = new Iterator[(TableBlockIndexUniqueIdentifier, BlockletIndexDetailsWithSchema)] {
       // in case of success, failure or cancellation clear memory and stop execution
-      context.addTaskCompletionListener { _ =>
-        reader.close()
-      }
+      CarbonToSparkAdapter.addTaskCompletionListener(reader.close())

Review comment:
       here also i can see same code in 3.1 and 2.4, so why this required?

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -439,10 +445,31 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
     }
   }
 
+  private static long createTimeInstant(String dimensionValue, String dateFormat) {
+    String updatedDim = dimensionValue;
+    if (!dimensionValue.trim().contains(" ")) {
+      updatedDim += " 00:00:00";

Review comment:
       please add a comment here why we are adding the string and then add a TODO to analyze it and change it later once the integration is complete in alpha stage

##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -185,8 +224,390 @@ object CarbonToSparkAdapter {
   def getHiveExternalCatalog(sparkSession: SparkSession): HiveExternalCatalog = {
     sparkSession.sessionState.catalog.externalCatalog.asInstanceOf[HiveExternalCatalog]
   }
+
+  def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]) = {
+    FilePartition(index, files.toArray.toSeq)
+  }
+
+  def stringToTimestamp(timestamp: String): Option[Long] = {
+    DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp))
+  }
+
+  def getTableIdentifier(u: UnresolvedRelation): Some[TableIdentifier] = {
+    Some(u.tableIdentifier)
+  }
+
+  def dateToString(date: Int): String = {
+    DateTimeUtils.dateToString(date.toString.toInt)
+  }
+
+  def timeStampToString(timeStamp: Long): String = {
+    DateTimeUtils.timestampToString(timeStamp)
+  }
+
+  def stringToTime(value: String): java.util.Date = {
+    DateTimeUtils.stringToTime(value)
+  }
+
+  def getProcessingTime: String => Trigger = {
+    Trigger.ProcessingTime
+  }
+
+  def addTaskCompletionListener[U](f: => U) {
+    TaskContext.get().addTaskCompletionListener { context =>
+      f
+    }
+  }
+
+  def createShuffledRowRDD(sparkContext: SparkContext, localTopK: RDD[InternalRow],
+      child: SparkPlan, serializer: Serializer): ShuffledRowRDD = {
+    new ShuffledRowRDD(
+      ShuffleExchangeExec.prepareShuffleDependency(
+        localTopK, child.output, SinglePartition, serializer))
+  }
+
+  def getInsertIntoCommand(table: LogicalPlan,
+      partition: Map[String, Option[String]],
+      query: LogicalPlan,
+      overwrite: Boolean,
+      ifPartitionNotExists: Boolean): InsertIntoTable = {
+    InsertIntoTable(
+      table,
+      partition,
+      query,
+      overwrite,
+      ifPartitionNotExists)
+  }
+
+  def getExplainCommandObj(logicalPlan: LogicalPlan = OneRowRelation(),
+      mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(logicalPlan, mode.isDefined)
+  }
+
+  def getExplainCommandObj(mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(OneRowRelation(), mode.isDefined)
+  }
+
+  def invokeAnalyzerExecute(analyzer: Analyzer,
+      plan: LogicalPlan): LogicalPlan = {
+    analyzer.executeAndCheck(plan)
+  }
+
+  def normalizeExpressions(r: NamedExpression, attrs: AttributeSeq): NamedExpression = {
+    QueryPlan.normalizeExprId(r, attrs)
+  }
+
+  def getBuildRight: BuildSide = {
+    BuildRight
+  }
+
+  def getBuildLeft: BuildSide = {
+    BuildLeft
+  }
+
+  type CarbonBuildSideType = BuildSide
+  type InsertIntoStatementWrapper = InsertIntoTable
+
+  def withNewExecutionId[T](sparkSession: SparkSession, queryExecution: QueryExecution): T => T = {
+    SQLExecution.withNewExecutionId(sparkSession, queryExecution)(_)
+  }
+
+  def getTableIdentifier(parts: TableIdentifier): TableIdentifier = {
+    parts
+  }
+
+  def createJoinNode(child: LogicalPlan,
+      targetTable: LogicalPlan,
+      joinType: JoinType,
+      condition: Option[Expression]): Join = {
+    Join(child, targetTable, joinType, condition)
+  }
+
+  def getPartitionsFromInsert(x: InsertIntoStatementWrapper): Map[String, Option[String]] = {
+    x.partition
+  }
+
+  def getStatisticsObj(outputList: Seq[NamedExpression],

Review comment:
       i think this method no one using, can remove if no one uses

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/test/SparkTestQueryExecutor.scala
##########
@@ -71,6 +71,12 @@ object SparkTestQueryExecutor {
     .config("spark.sql.warehouse.dir", warehouse)
     .config("spark.sql.crossJoin.enabled", "true")
     .config("spark.sql.extensions", extensions)
+    .config("spark.sql.storeAssignmentPolicy", "legacy")
+    .config("spark.sql.legacy.timeParserPolicy", "legacy")

Review comment:
       add TODO to writs test cases for non legacy

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -229,8 +270,388 @@ object CarbonToSparkAdapter {
       .unwrapped
       .asInstanceOf[HiveExternalCatalog]
   }
+
+  def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]): FilePartition = {
+    FilePartition(index, files.toArray)
+  }
+
+  def stringToTimestamp(timestamp: String): Option[Long] = {
+    DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp))
+  }
+
+  def stringToTime(value: String): java.util.Date = {
+    DateTimeUtils.stringToTime(value)
+  }
+
+  def timeStampToString(timeStamp: Long): String = {
+    DateTimeUtils.timestampToString(timeStamp)
+  }
+
+  def getTableIdentifier(u: UnresolvedRelation): Some[TableIdentifier] = {
+    Some(u.tableIdentifier)
+  }
+
+  def dateToString(date: Int): String = {
+    DateTimeUtils.dateToString(date.toString.toInt)
+  }
+
+  def getProcessingTime: String => Trigger = {
+    Trigger.ProcessingTime
+  }
+
+  def addTaskCompletionListener[U](f: => U) {
+    TaskContext.get().addTaskCompletionListener { context =>
+      f
+    }
+  }
+
+  def createShuffledRowRDD(sparkContext: SparkContext, localTopK: RDD[InternalRow],
+      child: SparkPlan, serializer: Serializer): ShuffledRowRDD = {
+    new ShuffledRowRDD(
+      ShuffleExchangeExec.prepareShuffleDependency(
+        localTopK, child.output, SinglePartition, serializer))
+  }
+
+  def getInsertIntoCommand(table: LogicalPlan,
+      partition: Map[String, Option[String]],
+      query: LogicalPlan,
+      overwrite: Boolean,
+      ifPartitionNotExists: Boolean): InsertIntoTable = {
+    InsertIntoTable(
+      table,
+      partition,
+      query,
+      overwrite,
+      ifPartitionNotExists)
+  }
+
+  def getExplainCommandObj(logicalPlan: LogicalPlan = OneRowRelation(),
+      mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(logicalPlan, mode.isDefined)
+  }
+
+  def invokeAnalyzerExecute(analyzer: Analyzer,
+      plan: LogicalPlan): LogicalPlan = {
+    analyzer.executeAndCheck(plan)
+  }
+
+  def normalizeExpressions(r: NamedExpression, attrs: AttributeSeq): NamedExpression = {
+    QueryPlan.normalizeExprId(r, attrs)
+  }
+
+  def getBuildRight: BuildSide = {
+    BuildRight
+  }
+
+  def getBuildLeft: BuildSide = {
+    BuildLeft
+  }
+
+  type CarbonBuildSideType = BuildSide
+  type InsertIntoStatementWrapper = InsertIntoTable
+
+  def withNewExecutionId[T](sparkSession: SparkSession, queryExecution: QueryExecution): T => T = {
+    SQLExecution.withNewExecutionId(sparkSession, queryExecution)(_)
+  }
+
+  def createJoinNode(child: LogicalPlan,
+      targetTable: LogicalPlan,
+      joinType: JoinType,
+      condition: Option[Expression]): Join = {
+    Join(child, targetTable, joinType, condition)
+  }
+
+  def getPartitionsFromInsert(x: InsertIntoStatementWrapper): Map[String, Option[String]] = {
+    x.partition
+  }
+
+  def getTableIdentifier(parts: TableIdentifier): TableIdentifier = {
+    parts
+  }
+
+  def getStatisticsObj(outputList: Seq[NamedExpression],
+      plan: LogicalPlan, stats: Statistics,
+      aliasMap: Option[AttributeMap[Attribute]] = None): Statistics = {
+    val output = outputList.map(_.toAttribute)
+    val mapSeq = plan.collect { case n: logical.LeafNode => n }.map {
+      table => AttributeMap(table.output.zip(output))
+    }
+    val rewrites = mapSeq.head
+    val attributes: AttributeMap[ColumnStat] = stats.attributeStats
+    var attributeStats = AttributeMap(attributes.iterator
+      .map { pair => (rewrites(pair._1), pair._2) }.toSeq)
+    if (aliasMap.isDefined) {
+      attributeStats = AttributeMap(
+        attributeStats.map(pair => (aliasMap.get(pair._1), pair._2)).toSeq)
+    }
+    Statistics(stats.sizeInBytes, stats.rowCount, attributeStats, stats.hints)
+  }
+
+  def createRefreshTableCommand(tableIdentifier: TableIdentifier): RefreshTable = {
+    RefreshTable(tableIdentifier)
+  }
+
+  type RefreshTables = RefreshTable

Review comment:
       as observed so many common code present for 2.3 and 3.1, so please do the refactor as given in previous comment

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/CarbonBoundReference.scala
##########
@@ -42,4 +42,4 @@ case class CarbonBoundReference(colExp: ColumnExpression, dataType: DataType, nu
   override def qualifier: Seq[String] = null
 
   override def newInstance(): NamedExpression = throw new UnsupportedOperationException
-}
\ No newline at end of file
+}

Review comment:
       revert this

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.spark.sql.execution.strategy
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.CarbonInputMetrics
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.CarbonDatasourceHadoopRelation
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonSparkDataSourceUtil
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, SortOrder, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.{Expression => SparkExpression}
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
+import org.apache.spark.sql.execution.{ColumnarBatchScan, DataSourceScanExec, WholeStageCodegenExec}
+import org.apache.spark.sql.optimizer.CarbonFilters
+import org.apache.spark.sql.types.AtomicType
+
+import org.apache.carbondata.core.index.IndexFilter
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.metadata.schema.BucketingInfo
+import org.apache.carbondata.core.readcommitter.ReadCommittedScope
+import org.apache.carbondata.core.scan.expression.Expression
+import org.apache.carbondata.core.scan.expression.logical.AndExpression
+import org.apache.carbondata.hadoop.CarbonProjection
+import org.apache.carbondata.spark.rdd.CarbonScanRDD
+
+/**
+ *  Physical plan node for scanning data. It is applied for both tables
+ *  USING carbondata and STORED AS carbondata.
+ */
+case class CarbonDataSourceScan(

Review comment:
       this class has lot of duplicate code of 2.3, please check and refactor

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/CarbonAnalyzer.scala
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.analysis.Analyzer
+import org.apache.spark.sql.catalyst.catalog.SessionCatalog
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.CarbonReflectionUtils
+
+class CarbonAnalyzer(catalog: SessionCatalog,

Review comment:
       this is same for 2.3 and 2.4, please refactor to common package

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.spark.sql.hive.execution.command
+
+import org.apache.spark.sql.{CarbonEnv, CarbonToSparkAdapter, Row, SparkSession}

Review comment:
       same as 2.3, please move to common

##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/util/ExampleUtils.scala
##########
@@ -93,6 +93,7 @@ object ExampleUtils {
       .config("spark.driver.host", "localhost")
       .config("spark.sql.crossJoin.enabled", "true")
       .config("spark.sql.extensions", "org.apache.spark.sql.CarbonExtensions")
+      .config("spark.sql.legacy.timeParserPolicy", "LEGACY")

Review comment:
       please add a comment here to analyze and take care and add tests for other than LEGACY

##########
File path: integration/flink/src/test/scala/org/apache/carbon/flink/TestCarbonPartitionWriter.scala
##########
@@ -111,14 +111,13 @@ class TestCarbonPartitionWriter extends QueryTest with BeforeAndAfterAll {
         s"(SELECT * FROM $tableName" + "_segments)").collect()
       assert(rows.length > 0)
       for (index <- 0 until unloadedStageCount) {
-        val row = rows(index)
         assert(rows(index).getString(0) == null)
         assert(rows(index).getString(1).equals("Unload"))
         assert(rows(index).getString(2) != null)
-        assert(rows(index).getLong(3) == -1)
+        assert(rows(index).getString(3) == "-1")

Review comment:
       why we need to change here? can you please explain the reason behind

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
##########
@@ -183,7 +183,7 @@ object DataLoadProcessorStepOnSpark {
     val rowConverter = new RowConverterImpl(conf.getDataFields, conf, badRecordLogger)
     rowConverter.initialize()
 
-    TaskContext.get().addTaskCompletionListener { context =>
+    CarbonToSparkAdapter.addTaskCompletionListener {

Review comment:
       here also i don't see any diff between 2.4 and 3.1, why refactored, can you please explain

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
##########
@@ -321,7 +321,7 @@ object DeleteExecution {
           deleteStatus = SegmentStatus.SUCCESS
         } catch {
           case e : MultipleMatchingException =>
-            LOGGER.error(e.getMessage)
+          LOGGER.error(e.getMessage)

Review comment:
       revert this

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,735 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener, SessionCatalog}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, ExprId, Expression, ExpressionSet, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{JoinType, QueryPlan, logical}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode}
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, SQLExecution, ShuffledRowRDD, SimpleMode, SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, RefreshTableCommand, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FilePartition, PartitionedFile}
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.metric.SQLShuffleWriteMetricsReporter
+import org.apache.spark.sql.execution.strategy.CarbonDataSourceScan
+import org.apache.spark.sql.hive.HiveExternalCatalog
+import org.apache.spark.sql.internal.{SessionState, SharedState}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.parser.CarbonSparkSqlParserUtil.{checkIfDuplicateColumnExists, convertDbNameToLowerCase, validateStreamingProperty}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.streaming.Trigger
+import org.apache.spark.sql.types.{AbstractDataType, CharType, DataType, Metadata, StringType, StructField, VarcharType}
+import org.apache.spark.sql.util.SparkSQLUtil
+import org.apache.spark.unsafe.types.UTF8String
+
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.SchemaReader
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+import org.apache.carbondata.mv.plans.modular.{GroupBy, ModularPlan, Select}
+import org.apache.carbondata.spark.CarbonOption
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+object CarbonToSparkAdapter {
+
+  def addSparkSessionListener(sparkSession: SparkSession): Unit = {
+    sparkSession.sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        CarbonEnv.carbonEnvMap.remove(sparkSession)
+        ThreadLocalSessionInfo.unsetAll()
+      }
+    })
+  }
+
+  def addSparkListener(sparkContext: SparkContext): Unit = {
+    sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        SparkSession.setDefaultSession(null)
+      }
+    })
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Option[String],
+      attrRef : NamedExpression = null): AttributeReference = {
+    val qf = if (qualifier.nonEmpty) Seq(qualifier.get) else Seq.empty
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qf)
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Seq[String]): AttributeReference = {
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qualifier)
+  }
+
+  def lowerCaseAttribute(expression: Expression): Expression = expression.transform {
+    case attr: AttributeReference =>
+      CarbonToSparkAdapter.createAttributeReference(
+        attr.name.toLowerCase,
+        attr.dataType,
+        attr.nullable,
+        attr.metadata,
+        attr.exprId,
+        attr.qualifier)
+  }
+
+  def createAttributeReference(attr: AttributeReference,
+      attrName: String,
+      newSubsume: String): AttributeReference = {
+    AttributeReference(attrName, attr.dataType)(
+      exprId = attr.exprId,
+      qualifier = newSubsume.split("\n").map(_.trim))
+  }
+
+  def createScalaUDF(s: ScalaUDF, reference: AttributeReference): ScalaUDF = {
+    s.copy(children = Seq(reference))
+  }
+
+  def createExprCode(code: String, isNull: String, value: String, dataType: DataType): ExprCode = {
+    ExprCode(
+      code"$code",
+      JavaCode.isNullVariable(isNull),
+      JavaCode.variable(value, dataType))
+  }
+
+  def createAliasRef(
+      child: Expression,
+      name: String,
+      exprId: ExprId = NamedExpression.newExprId,
+      qualifier: Seq[String] = Seq.empty,
+      explicitMetadata: Option[Metadata] = None,
+      namedExpr: Option[NamedExpression] = None) : Alias = {

Review comment:
       looks like `namedExpr` is not used, please check in other files and remove

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowSegmentsAsSelectCommand.scala
##########
@@ -46,12 +49,25 @@ case class CarbonShowSegmentsAsSelectCommand(
     Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
     CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
   }
-  private lazy val df = createDataFrame
+
+  val tempViewName: String = makeTempViewName(carbonTable)
+
+  val df: DataFrame = createDataFrame
 
   override def output: Seq[Attribute] = {
-    df.queryExecution.analyzed.output.map { attr =>
-      AttributeReference(attr.name, attr.dataType, nullable = true)()
+    var attrList: Seq[Attribute] = Seq()

Review comment:
       this Seq should be declared at class level if output called multiple times? @kunal642 @vikramahuja1001 

##########
File path: index/examples/pom.xml
##########
@@ -81,9 +81,6 @@
   <profiles>
     <profile>
       <id>spark-2.3</id>
-      <activation>
-        <activeByDefault>true</activeByDefault>
-      </activation>

Review comment:
       do we need to add new profile here and make that active by default? @kunal642 @vikramahuja1001 

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkSqlAdapter.scala
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.spark.sql
+
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.{Attribute, EmptyRow, Expression}
+import org.apache.spark.sql.execution.FileSourceScanExec
+import org.apache.spark.sql.execution.datasources.HadoopFsRelation
+import org.apache.spark.sql.types.StructType
+
+object SparkSqlAdapter {
+
+  def initSparkSQL(): Unit = {
+  }
+
+  def getScanForSegments(
+      @transient relation: HadoopFsRelation,

Review comment:
       i dont see any difference between all profiles, please check and remove

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonBoundReference.scala
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.spark.sql
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, ExprId, LeafExpression, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.types.DataType
+
+import org.apache.carbondata.core.scan.expression.ColumnExpression
+
+case class CarbonBoundReference(colExp: ColumnExpression, dataType: DataType, nullable: Boolean)

Review comment:
       i can see this class same to all profiles, what is the change here?

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
##########
@@ -181,7 +182,7 @@ object CarbonSparkUtil {
    */
   def createHadoopJob(conf: Configuration = FileFactory.getConfiguration): Job = {
     val jobConf = new JobConf(conf)
-    SparkHadoopUtil.get.addCredentials(jobConf)
+    SparkUtil.addCredentials(jobConf)

Review comment:
       no need to add method for just one line, and only its called from here,so can revert this change I feel

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.
+ */

Review comment:
       same as 2.3, please move

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/MergeProjection.scala
##########
@@ -82,10 +83,10 @@ case class MergeProjection(
       if (output.contains(null)) {
         throw new CarbonMergeDataSetException(s"Not all columns are mapped")
       }
-      (new InterpretedMutableProjection(output ++ Seq(
+      (output ++ Seq(
         ds.queryExecution.analyzed.resolveQuoted(statusCol,
           sparkSession.sessionState.analyzer.resolver).get),
-        ds.queryExecution.analyzed.output), expectOutput)

Review comment:
       why  `expectOutput` is removed here?

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateDataSourceTableCommand.scala
##########
@@ -78,10 +78,8 @@ case class CarbonCreateDataSourceTableCommand(
       catalogTable.partitionColumnNames,
       caseSensitiveAnalysis)
     val rows = try {
-      CreateDataSourceTableCommand(
-        catalogTable,
-        ignoreIfExists
-      ).run(sparkSession)
+      org.apache.spark.sql.execution.CreateDataSourceTableCommand
+        .createDataSource(catalogTable, ignoreIfExists, sparkSession)

Review comment:
       i can see same code of `CreateDataSourceTableCommand` in both 3.1 and 2.4, so better to make a common package and make it clean instead of duplicating, we had followed the same when we did previous upgrades also

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,735 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener, SessionCatalog}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, ExprId, Expression, ExpressionSet, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{JoinType, QueryPlan, logical}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode}
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, SQLExecution, ShuffledRowRDD, SimpleMode, SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, RefreshTableCommand, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FilePartition, PartitionedFile}
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.metric.SQLShuffleWriteMetricsReporter
+import org.apache.spark.sql.execution.strategy.CarbonDataSourceScan
+import org.apache.spark.sql.hive.HiveExternalCatalog
+import org.apache.spark.sql.internal.{SessionState, SharedState}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.parser.CarbonSparkSqlParserUtil.{checkIfDuplicateColumnExists, convertDbNameToLowerCase, validateStreamingProperty}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.streaming.Trigger
+import org.apache.spark.sql.types.{AbstractDataType, CharType, DataType, Metadata, StringType, StructField, VarcharType}
+import org.apache.spark.sql.util.SparkSQLUtil
+import org.apache.spark.unsafe.types.UTF8String
+
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.SchemaReader
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+import org.apache.carbondata.mv.plans.modular.{GroupBy, ModularPlan, Select}
+import org.apache.carbondata.spark.CarbonOption
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+object CarbonToSparkAdapter {

Review comment:
       please pull out the common code between 2.4 and 3.1 and move to common package and avoid code duplication 

##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneIndexExample.scala
##########
@@ -60,40 +60,6 @@ object LuceneIndexExample {
          | AS 'lucene'
       """.stripMargin)
 
-    // 1. Compare the performance:
-
-    def time(code: => Unit): Double = {
-      val start = System.currentTimeMillis()
-      code
-      // return time in second
-      (System.currentTimeMillis() - start).toDouble / 1000
-    }
-
-    val timeWithoutLuceneIndex = time {
-
-      spark.sql(
-        s"""
-           | SELECT count(*)
-           | FROM personTable where id like '% test1 %'
-      """.stripMargin).show()
-
-    }
-
-    val timeWithLuceneIndex = time {
-
-      spark.sql(

Review comment:
       why this code removed?

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,734 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}

Review comment:
       the import order doesn't comply to scalastyle rules, please correct it and I wonder how the CI is passing with style errors, please check the new CI and correct it and make sure to check all the new fils once for style issues, now it will directly give the errors in red color

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/joins/BroadCastPolygonFilterPushJoin.scala
##########
@@ -47,16 +45,13 @@ case class BroadCastPolygonFilterPushJoin(
     leftKeys: Seq[Expression],
     rightKeys: Seq[Expression],
     joinType: JoinType,
-    buildSide: BuildSide,
     condition: Option[Expression],
     left: SparkPlan,
     right: SparkPlan
-) extends BinaryExecNode with HashJoin {

Review comment:
       why we removed `HashJoin` here? 

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SICreationCommand.scala
##########
@@ -704,5 +705,13 @@ private[sql] case class CarbonCreateSecondaryIndexCommand(
     columnSchema
   }
 
+  override def clone(): LogicalPlan = {
+    CarbonCreateSecondaryIndexCommand(indexModel,

Review comment:
       why this is required?

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
##########
@@ -501,40 +321,6 @@ object CarbonSparkSqlParserUtil {
     }
   }
 
-  /**
-   * Validates the partition columns and return's A tuple of partition columns and partitioner
-   * fields.
-   *
-   * @param partitionColumns        An instance of ColTypeListContext having parser rules for
-   *                                column.
-   * @param colNames                <Seq[String]> Sequence of Table column names.
-   * @param tableProperties         <Map[String, String]> Table property map.
-   * @param partitionByStructFields Seq[StructField] Sequence of partition fields.
-   * @return <Seq[PartitionerField]> A Seq of partitioner fields.
-   */
-  def validatePartitionFields(

Review comment:
       this method and above create table looks same in both 3.1 and 2.4, is small diff, we will do proper refactoring, if common to 2.4 and 3.1, as give comment in one of the previous comments, we will add a common package and do a clean refactor

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
##########
@@ -604,11 +605,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
 
 
   protected lazy val alterTableColumnRenameAndModifyDataType: Parser[LogicalPlan] =
-    ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ CHANGE ~ ident ~ ident ~
-    ident ~ opt("(" ~> rep1sep(valueOptions, ",") <~ ")") <~ opt(";") ^^ {
-      case dbName ~ table ~ change ~ columnName ~ columnNameCopy ~ dataType ~ values =>
+    ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ (CHANGE ~> ident) ~ ident ~ ident ~

Review comment:
       please update the ddl in document and add an example with comment also and add test cases with comment. Now we change comment from properties

##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -185,8 +224,390 @@ object CarbonToSparkAdapter {
   def getHiveExternalCatalog(sparkSession: SparkSession): HiveExternalCatalog = {
     sparkSession.sessionState.catalog.externalCatalog.asInstanceOf[HiveExternalCatalog]
   }
+
+  def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]) = {
+    FilePartition(index, files.toArray.toSeq)
+  }
+
+  def stringToTimestamp(timestamp: String): Option[Long] = {
+    DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp))
+  }
+
+  def getTableIdentifier(u: UnresolvedRelation): Some[TableIdentifier] = {
+    Some(u.tableIdentifier)
+  }
+
+  def dateToString(date: Int): String = {
+    DateTimeUtils.dateToString(date.toString.toInt)
+  }
+
+  def timeStampToString(timeStamp: Long): String = {
+    DateTimeUtils.timestampToString(timeStamp)
+  }

Review comment:
       in these, many methods i can see duplicate to 2.4,please refactor as gave comment before

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/util/CreateTableCommonUtil.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.spark.sql.util
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogUtils}
+import org.apache.spark.sql.execution.datasources.{DataSource, HadoopFsRelation}
+import org.apache.spark.sql.internal.SessionState
+import org.apache.spark.sql.sources.BaseRelation
+import org.apache.spark.sql.types.StructType
+
+object CreateTableCommonUtil {
+
+  def getNewTable(sparkSession: SparkSession, sessionState: SessionState,

Review comment:
       add comment for method and rename to getCatalogtable

##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala
##########
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.hive.execution.command
 
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.{CarbonEnv, CarbonToSparkAdapter, Row, SparkSession}

Review comment:
       revert if not required

##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/execution/CarbonCodegenSupport.scala
##########
@@ -14,15 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.spark.sql.execution
 
-package org.apache.carbondata.spark.adapter
+import org.apache.spark.sql.execution.joins.HashJoin
 
-import scala.collection.mutable.ArrayBuffer
+trait CarbonCodegenSupport extends SparkPlan with HashJoin {

Review comment:
       add comment for trait, why its required. Since here already extends Hashjoin, I think implementation no need to add with hashjoin, please check once

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, CatalogUtils}
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.util.CreateTableCommonUtil.getNewTable
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+
+case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boolean)
+  extends RunnableCommand {
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    assert(table.tableType != CatalogTableType.VIEW)
+    assert(table.provider.isDefined)
+    val sessionState = sparkSession.sessionState
+    if (sessionState.catalog.tableExists(table.identifier)) {
+      if (ignoreIfExists) {
+        return Seq.empty[Row]
+      } else {
+        throw new AnalysisException(s"Table ${table.identifier.unquotedString} already exists.")
+      }
+    }
+    val newTable: CatalogTable = getNewTable(sparkSession, sessionState, table, LOGGER)
+
+    // We will return Nil or throw exception at the beginning if the table already exists, so when
+    // we reach here, the table should not exist and we should set `ignoreIfExists` to false.
+    sessionState.catalog.createTable(newTable, ignoreIfExists = false, validateLocation = false)
+    Seq.empty[Row]
+  }
+}
+
+object CreateDataSourceTableCommand {

Review comment:
       i think this can be combined with 2.3

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/CarbonSIRebuildRDD.scala
##########
@@ -214,9 +214,7 @@ class CarbonSIRebuildRDD[K, V](
           new SparkDataTypeConverterImpl)
 
         // add task completion listener to clean up the resources
-        context.addTaskCompletionListener { _ =>
-          close()
-        }
+        CarbonToSparkAdapter.addTaskCompletionListener(close())

Review comment:
       same as old comment

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/execution/CarbonCodegenSupport.scala
##########
@@ -0,0 +1,23 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.spark.sql.execution.joins.HashJoin
+
+trait CarbonCodegenSupport extends SparkPlan with HashJoin {

Review comment:
       Same comment as this class in 2.3, also please make common package and keep there as no change in 2.3 and 2.4

##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+
+object CreateDataSourceTableCommand {

Review comment:
       remove this from here and add to `CarbonToSparkAdapter` similar to other profiles

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.parser.ParserUtils.string
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{AddTableColumnsContext, CreateHiveTableContext}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkSqlAstBuilder
+import org.apache.spark.sql.internal.SQLConf

Review comment:
       this is same for 2.3 and 2.4, please move to common package

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.CarbonToSparkAdapter

Review comment:
       this is same as 2.3, please refactor and move to common

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.spark.sql.parser
+

Review comment:
       same as 2.3, please move to common

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/avro/AvroFileFormatFactory.scala
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.spark.sql.avro

Review comment:
       this is same code as 2.4, please move to common package

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/execution/CarbonCodegenSupport.scala
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
+import org.apache.spark.sql.execution.joins.HashJoin
+
+trait CarbonCodegenSupport extends SparkPlan with HashJoin {
+
+  // TODO: Spark has started supporting Codegen for Join, Carbon needs to implement the same.
+  override def supportCodegen: Boolean = false

Review comment:
       can create a subjira under this and track it

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, CatalogUtils}
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.util.CreateTableCommonUtil.getNewTable
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+
+case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boolean)
+  extends RunnableCommand {
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    assert(table.tableType != CatalogTableType.VIEW)
+    assert(table.provider.isDefined)
+
+    val sessionState = sparkSession.sessionState
+    if (sessionState.catalog.tableExists(table.identifier)) {
+      if (ignoreIfExists) {
+        return Seq.empty[Row]
+      } else {
+        throw new AnalysisException(s"Table ${table.identifier.unquotedString} already exists.")
+      }

Review comment:
       i can see same code for 2.4 and 3.1 please refactor and add extra line at end of file, its a style issue, please check why new CI is not finding it




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-851489591


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655309875



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowSegmentsAsSelectCommand.scala
##########
@@ -46,12 +49,25 @@ case class CarbonShowSegmentsAsSelectCommand(
     Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
     CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
   }
-  private lazy val df = createDataFrame
+
+  val tempViewName: String = makeTempViewName(carbonTable)
+
+  val df: DataFrame = createDataFrame
 
   override def output: Seq[Attribute] = {
-    df.queryExecution.analyzed.output.map { attr =>
-      AttributeReference(attr.name, attr.dataType, nullable = true)()
+    var attrList: Seq[Attribute] = Seq()

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-850337778


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-864942250


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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-856527816


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-854538244


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r648161768



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/carbondata/execution/datasources/SparkCarbonFileFormat.scala
##########
@@ -86,6 +87,11 @@ class SparkCarbonFileFormat extends FileFormat
       options: Map[String, String],
       files: Seq[FileStatus]): Option[StructType] = {
     val conf = sparkSession.sessionState.newHadoopConf()
+    if (options.isEmpty && files.isEmpty) {

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647102683



##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonBoundReference.scala
##########
@@ -18,8 +18,8 @@
 package org.apache.spark.sql
 
 import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
 import org.apache.spark.sql.catalyst.expressions.{Attribute, ExprId, LeafExpression, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
 import org.apache.spark.sql.types.DataType
 

Review comment:
       Needed or else scalastyle fails




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644563223



##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/CarbonAnalyzer.scala
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.analysis.Analyzer
+import org.apache.spark.sql.catalyst.catalog.SessionCatalog
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.CarbonReflectionUtils
+
+class CarbonAnalyzer(catalog: SessionCatalog,

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646318477



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/carbondata/execution/datasources/SparkCarbonFileFormat.scala
##########
@@ -86,6 +87,11 @@ class SparkCarbonFileFormat extends FileFormat
       options: Map[String, String],
       files: Seq[FileStatus]): Option[StructType] = {
     val conf = sparkSession.sessionState.newHadoopConf()
+    if (options.isEmpty && files.isEmpty) {

Review comment:
       remove this code




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-851913152


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-857654759


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


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

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



[GitHub] [carbondata] akashrn5 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-848059100


   @vikramahuja1001 please update the description with more information of what and all changes included in 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:
users@infra.apache.org



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644537009



##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -2644,5 +2644,4 @@ private CarbonCommonConstants() {
   public static final String CARBON_MAP_ORDER_PUSHDOWN = "carbon.mapOrderPushDown";
 
   public static final String CARBON_SDK_EMPTY_METADATA_PATH = "emptyMetadataFolder";
-

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r648161768



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/carbondata/execution/datasources/SparkCarbonFileFormat.scala
##########
@@ -86,6 +87,11 @@ class SparkCarbonFileFormat extends FileFormat
       options: Map[String, String],
       files: Seq[FileStatus]): Option[StructType] = {
     val conf = sparkSession.sessionState.newHadoopConf()
+    if (options.isEmpty && files.isEmpty) {

Review comment:
       done, changed exceptions




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r645302566



##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, CatalogUtils}
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.util.CreateTableCommonUtil.getNewTable
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+
+case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boolean)
+  extends RunnableCommand {
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    assert(table.tableType != CatalogTableType.VIEW)
+    assert(table.provider.isDefined)
+    val sessionState = sparkSession.sessionState
+    if (sessionState.catalog.tableExists(table.identifier)) {
+      if (ignoreIfExists) {
+        return Seq.empty[Row]
+      } else {
+        throw new AnalysisException(s"Table ${table.identifier.unquotedString} already exists.")
+      }
+    }
+    val newTable: CatalogTable = getNewTable(sparkSession, sessionState, table, LOGGER)
+
+    // We will return Nil or throw exception at the beginning if the table already exists, so when
+    // we reach here, the table should not exist and we should set `ignoreIfExists` to false.
+    sessionState.catalog.createTable(newTable, ignoreIfExists = false, validateLocation = false)
+    Seq.empty[Row]
+  }
+}
+
+object CreateDataSourceTableCommand {

Review comment:
       It is common to 3.1, combined it with 3.1




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647119804



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkVersionAdapter.scala
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener, SessionCatalog}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, Expression, ExpressionSet, ExprId, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode}
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, ShuffledRowRDD, SimpleMode, SparkPlan, SQLExecution, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, RefreshTableCommand, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.metric.SQLShuffleWriteMetricsReporter
+import org.apache.spark.sql.execution.strategy.CarbonDataSourceScan
+import org.apache.spark.sql.internal.{SessionState, SharedState}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.parser.CarbonSparkSqlParserUtil.{checkIfDuplicateColumnExists, convertDbNameToLowerCase, validateStreamingProperty}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{AbstractDataType, CharType, DataType, Metadata, StringType, StructField, VarcharType}
+import org.apache.spark.sql.util.SparkSQLUtil
+import org.apache.spark.unsafe.types.UTF8String
+
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.SchemaReader
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+import org.apache.carbondata.spark.CarbonOption
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+object SparkVersionAdapter {
+
+  def getExplainCommandObj(logicalPlan: LogicalPlan = OneRowRelation(),
+                           mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(logicalPlan, ExplainMode.fromString(mode.getOrElse(SimpleMode.name)))
+  }
+
+  def getExplainCommandObj(mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(OneRowRelation(), ExplainMode.fromString(mode.getOrElse(SimpleMode.name)))
+  }
+
+  /**
+   * As a part of SPARK-24085 Hive tables supports scala subquery for
+   * the partitioned tables,so Carbon also needs to supports
+   * @param partitionSet
+   * @param filterPredicates
+   * @return
+   */
+  def getPartitionFilter(
+                          partitionSet: AttributeSet,
+                          filterPredicates: Seq[Expression]): Seq[Expression] = {
+    filterPredicates
+      .filterNot(SubqueryExpression.hasSubquery)
+      .filter { filter =>
+        filter.references.nonEmpty && filter.references.subsetOf(partitionSet)
+      }
+  }
+
+  def getDataFilter(partitionSet: AttributeSet, filter: Seq[Expression]): Seq[Expression] = {
+    filter
+  }
+
+  // As per SPARK-22520 OptimizeCodegen is removed in 2.3.1
+  def getOptimizeCodegenRule(): Seq[Rule[LogicalPlan]] = {
+    Seq.empty
+  }
+
+  def getUpdatedStorageFormat(storageFormat: CatalogStorageFormat,
+                              map: Map[String, String],
+                              tablePath: String): CatalogStorageFormat = {
+    storageFormat.copy(properties = map, locationUri = Some(new URI(tablePath)))
+  }
+
+  def getOutput(subQueryAlias: SubqueryAlias): Seq[Attribute] = {
+    val newAlias = Seq(subQueryAlias.identifier.name)
+    subQueryAlias.child.output.map(_.withQualifier(newAlias))
+  }
+
+  def stringToTimestamp(timestamp: String): Option[Long] = {
+    DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp), ZoneId.systemDefault())
+  }
+
+  def stringToTime(value: String): java.util.Date = {
+    stringToDateValue(value)
+  }
+
+  def getPredicate(inputSchema: Seq[Attribute],
+                   condition: Option[Expression]): InternalRow => Boolean = {
+    Predicate.create(condition.get, inputSchema).eval _
+  }
+
+  @tailrec
+  private def stringToDateValue(value: String): java.util.Date = {
+    val indexOfGMT = value.indexOf("GMT")
+    if (indexOfGMT != -1) {
+      // ISO8601 with a weird time zone specifier (2000-01-01T00:00GMT+01:00)
+      val s0 = value.substring(0, indexOfGMT)
+      val s1 = value.substring(indexOfGMT + 3)
+      // Mapped to 2000-01-01T00:00+01:00
+      stringToDateValue(s0 + s1)
+    } else if (!value.contains('T')) {
+      // JDBC escape string
+      if (value.contains(' ')) {
+        Timestamp.valueOf(value)
+      } else {
+        Date.valueOf(value)
+      }
+    } else {
+      DatatypeConverter.parseDateTime(value).getTime
+    }
+  }
+
+  def timeStampToString(timeStamp: Long): String = {
+    TimestampFormatter.getFractionFormatter(ZoneId.systemDefault()).format(timeStamp)
+  }
+
+  def dateToString(date: Int): String = {
+    DateTimeUtils.daysToLocalDate(date).toString
+  }
+
+  def addTaskCompletionListener[U](f: => U) {
+    TaskContext.get().addTaskCompletionListener[Unit] { context =>
+      f
+    }
+  }
+
+  def getTableIdentifier(u: UnresolvedRelation): Some[TableIdentifier] = {
+    val tableName = u.tableName.split("\\.")
+    if (tableName.size == 2) {
+      Some(TableIdentifier(tableName(1), Option(tableName(0))))
+    } else {
+      val currentDatabase = SparkSQLUtil.getSparkSession.sessionState.catalog.getCurrentDatabase
+      Some(TableIdentifier(tableName(0), Option(currentDatabase)))
+    }
+  }
+
+  def createRangeListScalaUDF(toRangeListUDF: ToRangeListAsStringUDF,
+                              dataType: StringType.type,
+                              children: Seq[Expression],
+                              inputTypes: Seq[DataType]): ScalaUDF = {
+    ScalaUDF(toRangeListUDF,
+      dataType,
+      children,
+      Nil,
+      None,
+      Some("ToRangeListAsString"))
+  }
+
+  def getTransformedPolygonJoinUdf(scalaUdf: ScalaUDF,
+                                   udfChildren: Seq[Expression],
+                                   polygonJoinUdf: InPolygonJoinUDF): ScalaUDF = {
+    ScalaUDF(polygonJoinUdf,
+      scalaUdf.dataType,
+      udfChildren,
+      scalaUdf.inputEncoders,
+      scalaUdf.outputEncoder,
+      scalaUdf.udfName)
+  }
+
+  def getTableIdentifier(parts: Seq[String]): TableIdentifier = {
+    if (parts.length == 1) {
+      TableIdentifier(parts.head, None)
+    } else {
+      TableIdentifier(parts(1), Option(parts.head))
+    }
+  }
+
+  def createShuffledRowRDD(sparkContext: SparkContext, localTopK: RDD[InternalRow],
+                           child: SparkPlan, serializer: Serializer): ShuffledRowRDD = {
+    val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext)
+    new ShuffledRowRDD(
+      ShuffleExchangeExec.prepareShuffleDependency(
+        localTopK, child.output, SinglePartition, serializer, writeMetrics), writeMetrics)
+  }
+
+  def invokeAnalyzerExecute(analyzer: Analyzer,
+                            plan: LogicalPlan): LogicalPlan = {
+    analyzer.executeAndCheck(plan, QueryPlanningTracker.get.getOrElse(new QueryPlanningTracker))
+  }
+
+  def normalizeExpressions[T <: Expression](r: T, attrs: AttributeSeq): T = {
+    QueryPlan.normalizeExpressions(r, attrs)
+  }
+
+  def getBuildRight: BuildSide = {
+    BuildRight
+  }
+
+  def getBuildLeft: BuildSide = {
+    BuildLeft
+  }
+
+  def withNewExecutionId[T](sparkSession: SparkSession, queryExecution: QueryExecution): T => T = {
+    SQLExecution.withNewExecutionId(queryExecution, None)(_)
+  }
+
+  def createJoinNode(child: LogicalPlan,
+                     targetTable: LogicalPlan,
+                     joinType: JoinType,
+                     condition: Option[Expression]): Join = {
+    Join(child, targetTable, joinType, condition, JoinHint.NONE)
+  }
+
+  def getPartitionsFromInsert(x: InsertIntoStatementWrapper): Map[String, Option[String]] = {
+    x.partitionSpec
+  }
+
+  type CarbonBuildSideType = BuildSide
+  type InsertIntoStatementWrapper = InsertIntoStatement
+
+  def createRefreshTableCommand(tableIdentifier: TableIdentifier): RefreshTableCommand = {
+    RefreshTableCommand(tableIdentifier)
+  }
+
+  type RefreshTables = RefreshTableCommand
+
+  /**
+   * Validates the partition columns and return's A tuple of partition columns and partitioner
+   * fields.
+   *
+   * @param partitionColumns        An instance of ColTypeListContext having parser rules for
+   *                                column.
+   * @param colNames                <Seq[String]> Sequence of Table column names.
+   * @param tableProperties         <Map[String, String]> Table property map.
+   * @param partitionByStructFields Seq[StructField] Sequence of partition fields.
+   * @return <Seq[PartitionerField]> A Seq of partitioner fields.
+   */
+  def validatePartitionFields(
+                               partitionColumns: PartitionFieldListContext,
+                               colNames: Seq[String],
+                               tableProperties: mutable.Map[String, String],
+                               partitionByStructFields: Seq[StructField]): Seq[PartitionerField] = {
+
+    val partitionerFields = partitionByStructFields.map { structField =>
+      PartitionerField(structField.name, Some(structField.dataType.toString), null)
+    }
+    // validate partition clause
+    if (partitionerFields.nonEmpty) {
+      // partition columns should not be part of the schema
+      val badPartCols = partitionerFields.map(_.partitionColumn.toLowerCase).toSet
+        .intersect(colNames.map(_.toLowerCase).toSet)
+      if (badPartCols.nonEmpty) {
+        operationNotAllowed(s"Partition columns should not be specified in the schema: " +
+          badPartCols.map("\"" + _ + "\"").mkString("[", ",", "]")
+          , partitionColumns: PartitionFieldListContext)
+      }
+    }
+    partitionerFields
+  }
+
+  /**
+   * The method validates the create table command and returns the create table or
+   * ctas table LogicalPlan.
+   *
+   * @param createTableTuple a tuple of (CreateTableHeaderContext, SkewSpecContext,
+   *                         BucketSpecContext, PartitionFieldListContext, ColTypeListContext,
+   *                         TablePropertyListContext,
+   *                         LocationSpecContext, Option[String], TerminalNode, QueryContext,
+   *                         String)
+   * @param extraTableTuple  A tuple of (Seq[StructField], Boolean, TableIdentifier, Boolean,
+   *                         Seq[String],
+   *                         Option[String], mutable.Map[String, String], Map[String, String],
+   *                         Seq[StructField],
+   *                         Seq[PartitionerField], CarbonSpark2SqlParser, SparkSession,
+   *                         Option[LogicalPlan])
+   * @return <LogicalPlan> of create table or ctas table
+   *
+   */
+  def createCarbonTable(createTableTuple: (CreateTableHeaderContext, SkewSpecContext,
+    BucketSpecContext, PartitionFieldListContext, ColTypeListContext, TablePropertyListContext,
+    LocationSpecContext, Option[String], TerminalNode, QueryContext, String),
+    extraTableTuple: (Seq[StructField], Boolean, TableIdentifier, Boolean, Seq[String],
+    Option[String], mutable.Map[String, String], Map[String, String], Seq[StructField],
+    Seq[PartitionerField], CarbonSpark2SqlParser, SparkSession,
+    Option[LogicalPlan])): LogicalPlan = {
+    val (tableHeader, skewSpecContext, bucketSpecContext, partitionColumns, columns,
+    tablePropertyList, locationSpecContext, tableComment, ctas, query, provider) = createTableTuple
+    val (cols, external, tableIdentifier, ifNotExists, colNames, tablePath,
+    tableProperties, properties, partitionByStructFields, partitionFields,
+    parser, sparkSession, selectQuery) = extraTableTuple
+    val options = new CarbonOption(properties)
+    // validate streaming property
+    validateStreamingProperty(options)
+    var fields = parser.getFields(cols ++ partitionByStructFields)
+    // validate for create table as select
+    selectQuery match {
+      case Some(q) =>
+        // create table as select does not allow creation of partitioned table
+        if (partitionFields.nonEmpty) {
+          val errorMessage = "A Create Table As Select (CTAS) statement is not allowed to " +
+            "create a partitioned table using Carbondata file formats."
+          operationNotAllowed(errorMessage, partitionColumns)
+        }
+        // create table as select does not allow to explicitly specify schema
+        if (fields.nonEmpty) {
+          operationNotAllowed(
+            "Schema may not be specified in a Create Table As Select (CTAS) statement", columns)
+        }
+        // external table is not allow
+        if (external) {
+          operationNotAllowed("Create external table as select", tableHeader)
+        }
+        fields = parser
+          .getFields(CarbonEnv.getInstance(sparkSession).carbonMetaStore
+            .getSchemaFromUnresolvedRelation(sparkSession, Some(q).get))
+      case _ =>
+      // ignore this case
+    }
+    val columnNames = fields.map(_.name.get)
+    checkIfDuplicateColumnExists(columns, tableIdentifier, columnNames)
+    if (partitionFields.nonEmpty && options.isStreaming) {
+      operationNotAllowed("Streaming is not allowed on partitioned table", partitionColumns)
+    }
+
+    if (!external && fields.isEmpty) {
+      throw new MalformedCarbonCommandException("Creating table without column(s) is not supported")
+    }
+    if (external && fields.isEmpty && tableProperties.nonEmpty) {
+      // as fields are always zero for external table, cannot validate table properties.
+      operationNotAllowed(
+        "Table properties are not supported for external table", tablePropertyList)
+    }
+
+    // Global dictionary is deprecated since 2.0
+    if (tableProperties.contains(CarbonCommonConstants.DICTIONARY_INCLUDE) ||
+      tableProperties.contains(CarbonCommonConstants.DICTIONARY_EXCLUDE)) {
+      DeprecatedFeatureException.globalDictNotSupported()
+    }
+
+    val bucketFields = parser.getBucketFields(tableProperties, fields, options)
+    var isTransactionalTable: Boolean = true
+
+    val tableInfo = if (external) {
+      if (fields.nonEmpty) {
+        // user provided schema for this external table, this is not allow currently
+        // see CARBONDATA-2866
+        operationNotAllowed(
+          "Schema must not be specified for external table", columns)
+      }
+      if (partitionByStructFields.nonEmpty) {
+        operationNotAllowed(
+          "Partition is not supported for external table", partitionColumns)
+      }
+      // read table info from schema file in the provided table path
+      // external table also must convert table name to lower case
+      val identifier = AbsoluteTableIdentifier.from(
+        tablePath.get,
+        CarbonEnv.getDatabaseName(tableIdentifier.database)(sparkSession).toLowerCase(),
+        tableIdentifier.table.toLowerCase())
+      val table = try {
+        val schemaPath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath)
+        if (!FileFactory.isFileExist(schemaPath)) {
+          if (provider.equalsIgnoreCase("'carbonfile'")) {
+            SchemaReader.inferSchema(identifier, true)
+          } else {
+            isTransactionalTable = false
+            SchemaReader.inferSchema(identifier, false)
+          }
+        } else {
+          SchemaReader.getTableInfo(identifier)
+        }
+      } catch {
+        case e: Throwable =>
+          operationNotAllowed(s"Invalid table path provided: ${ tablePath.get } ", tableHeader)
+      }
+
+      // set "_external" property, so that DROP TABLE will not delete the data
+      if (provider.equalsIgnoreCase("'carbonfile'")) {
+        table.getFactTable.getTableProperties.put("_filelevelformat", "true")
+        table.getFactTable.getTableProperties.put("_external", "false")
+      } else {
+        table.getFactTable.getTableProperties.put("_external", "true")
+        table.getFactTable.getTableProperties.put("_filelevelformat", "false")
+      }
+      var isLocalDic_enabled = table.getFactTable.getTableProperties
+        .get(CarbonCommonConstants.LOCAL_DICTIONARY_ENABLE)
+      if (null == isLocalDic_enabled) {
+        table.getFactTable.getTableProperties
+          .put(CarbonCommonConstants.LOCAL_DICTIONARY_ENABLE,
+            CarbonProperties.getInstance()
+              .getProperty(CarbonCommonConstants.LOCAL_DICTIONARY_SYSTEM_ENABLE,
+                CarbonCommonConstants.LOCAL_DICTIONARY_ENABLE_DEFAULT))
+      }
+      isLocalDic_enabled = table.getFactTable.getTableProperties
+        .get(CarbonCommonConstants.LOCAL_DICTIONARY_ENABLE)
+      if (CarbonScalaUtil.validateLocalDictionaryEnable(isLocalDic_enabled) &&
+        isLocalDic_enabled.toBoolean) {
+        val allColumns = table.getFactTable.getListOfColumns
+        for (i <- 0 until allColumns.size()) {
+          val cols = allColumns.get(i)
+          if (cols.getDataType == DataTypes.STRING || cols.getDataType == DataTypes.VARCHAR) {
+            cols.setLocalDictColumn(true)
+          }
+        }
+        table.getFactTable.setListOfColumns(allColumns)
+      }
+      table
+    } else {
+      // prepare table model of the collected tokens
+      val tableModel: TableModel = CarbonParserUtil.prepareTableModel(
+        ifNotExists,
+        convertDbNameToLowerCase(tableIdentifier.database),
+        tableIdentifier.table.toLowerCase,
+        fields,
+        partitionFields,
+        tableProperties,
+        bucketFields,
+        isAlterFlow = false,
+        tableComment)
+      TableNewProcessor(tableModel)
+    }
+    tableInfo.setTransactionalTable(isTransactionalTable)
+    selectQuery match {
+      case query@Some(q) =>
+        CarbonCreateTableAsSelectCommand(
+          tableInfo = tableInfo,
+          query = query.get,
+          ifNotExistsSet = ifNotExists,
+          tableLocation = tablePath)
+      case _ =>
+        CarbonCreateTableCommand(
+          tableInfo = tableInfo,
+          ifNotExistsSet = ifNotExists,
+          tableLocation = tablePath,
+          external)
+    }
+  }
+
+  def getField(parser: CarbonSpark2SqlParser,
+               schema: Seq[QualifiedColType], isExternal: Boolean = false): Seq[Field] = {
+    schema.map { col =>
+      parser.getFields(col.comment, col.name.head, col.dataType, isExternal)
+    }
+  }
+
+  def supportsBatchOrColumnar(scan: CarbonDataSourceScan): Boolean = {
+    scan.supportsColumnar
+  }
+
+  def createDataset(sparkSession: SparkSession, qe: QueryExecution) : Dataset[Row] = {
+    new Dataset[Row](qe, RowEncoder(qe.analyzed.schema))
+  }
+
+  def createSharedState(sparkContext: SparkContext) : SharedState = {
+    new SharedState(sparkContext, Map.empty[String, String])
+  }
+
+  def translateFilter(dataFilters: Seq[Expression]) : Seq[Filter] = {
+    dataFilters.flatMap(DataSourceStrategy.translateFilter(_,
+      supportNestedPredicatePushdown = false))
+  }
+
+  def getCarbonOptimizer(session: SparkSession,
+                         sessionState: SessionState): CarbonOptimizer = {
+    new CarbonOptimizer(session, sessionState.optimizer)
+  }
+
+  def isCharType(dataType: DataType): Boolean = {
+    dataType.isInstanceOf[CharType]
+  }
+
+  def isVarCharType(dataType: DataType): Boolean = {
+    dataType.isInstanceOf[VarcharType]
+  }
+
+  def getTypeName(s: AbstractDataType): String = {
+    s.defaultConcreteType.typeName
+  }
+
+  def getInsertIntoCommand(table: LogicalPlan,
+                           partition: Map[String, Option[String]],
+                           query: LogicalPlan,
+                           overwrite: Boolean,
+                           ifPartitionNotExists: Boolean): InsertIntoStatement = {
+    InsertIntoStatement(
+      table,
+      partition,
+      Nil,
+      query,
+      overwrite,
+      ifPartitionNotExists)
+  }
+}
+
+case class CarbonBuildSide(buildSide: BuildSide) {
+  def isRight: Boolean = buildSide.isInstanceOf[BuildRight.type]
+  def isLeft: Boolean = buildSide.isInstanceOf[BuildLeft.type]
+}
+
+abstract class CarbonTakeOrderedAndProjectExecHelper(sortOrder: Seq[SortOrder],
+    limit: Int, skipMapOrder: Boolean, readFromHead: Boolean) extends UnaryExecNode {
+  override def simpleString(maxFields: Int): String = {
+    val orderByString = sortOrder.mkString("[", ",", "]")
+    val outputString = output.mkString("[", ",", "]")
+
+    s"CarbonTakeOrderedAndProjectExec(limit=$limit, orderBy=$orderByString, " +
+      s"skipMapOrder=$skipMapOrder, readFromHead=$readFromHead, output=$outputString)"
+  }
+}

Review comment:
       done

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkVersionAdapter.scala
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener, SessionCatalog}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, Expression, ExpressionSet, ExprId, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode}
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, ShuffledRowRDD, SimpleMode, SparkPlan, SQLExecution, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, RefreshTableCommand, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.metric.SQLShuffleWriteMetricsReporter
+import org.apache.spark.sql.execution.strategy.CarbonDataSourceScan
+import org.apache.spark.sql.internal.{SessionState, SharedState}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.parser.CarbonSparkSqlParserUtil.{checkIfDuplicateColumnExists, convertDbNameToLowerCase, validateStreamingProperty}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{AbstractDataType, CharType, DataType, Metadata, StringType, StructField, VarcharType}
+import org.apache.spark.sql.util.SparkSQLUtil
+import org.apache.spark.unsafe.types.UTF8String
+
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.SchemaReader
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+import org.apache.carbondata.spark.CarbonOption
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+object SparkVersionAdapter {
+
+  def getExplainCommandObj(logicalPlan: LogicalPlan = OneRowRelation(),
+                           mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(logicalPlan, ExplainMode.fromString(mode.getOrElse(SimpleMode.name)))
+  }
+
+  def getExplainCommandObj(mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(OneRowRelation(), ExplainMode.fromString(mode.getOrElse(SimpleMode.name)))
+  }
+
+  /**
+   * As a part of SPARK-24085 Hive tables supports scala subquery for
+   * the partitioned tables,so Carbon also needs to supports
+   * @param partitionSet
+   * @param filterPredicates
+   * @return
+   */
+  def getPartitionFilter(
+                          partitionSet: AttributeSet,
+                          filterPredicates: Seq[Expression]): Seq[Expression] = {

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655308271



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
##########
@@ -1551,26 +1552,28 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
             sql("insert into hive_table select 'a','b','1'");
         }
 
-        val e2 = intercept[SparkException] {
-            sql("insert into hive_table2 select 'a','b','binary'");
-        }
+        if (!SparkUtil.isSparkVersionXAndAbove("3")) {
+            val e2 = intercept[SparkException] {
+                sql("insert into hive_table2 select 'a','b','binary'");
+            }
 
-        assert(e2.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e2.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))
 
-        val eInt2 = intercept[Exception] {
-            sql("insert into hive_table2 select 'a','b','1'");
-        }
+            val eInt2 = intercept[Exception] {
+                sql("insert into hive_table2 select 'a','b','1'");
+            }
 
-        val e3 = intercept[SparkException] {
-            sql("insert into parquet_table select 'a','b','binary'");
-        }
+            val e3 = intercept[SparkException] {
+                sql("insert into parquet_table select 'a','b','binary'");
+            }
 
-        assert(e3.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e3.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))
 
-        val eInt3 = intercept[Exception] {
-            sql("insert into parquet_table select 'a','b','1'");
+            val eInt3 = intercept[Exception] {
+                sql("insert into parquet_table select 'a','b','1'");
+            }

Review comment:
       if condition has been added, needed for that




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655305544



##########
File path: mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/Printers.scala
##########
@@ -358,6 +359,12 @@ trait Printers {
           ""
         }
         qualifierPrefix + quoteIdentifier(child.name) + " AS " + quoteIdentifier(a.name)
+      case a@Alias(child: AggregateExpression, _) =>
+        child.sql + " AS " + quoteIdentifier(a.name)

Review comment:
       done




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-865301616


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655295892



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,735 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener, SessionCatalog}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, ExprId, Expression, ExpressionSet, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{JoinType, QueryPlan, logical}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode}
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, SQLExecution, ShuffledRowRDD, SimpleMode, SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, RefreshTableCommand, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FilePartition, PartitionedFile}
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.metric.SQLShuffleWriteMetricsReporter
+import org.apache.spark.sql.execution.strategy.CarbonDataSourceScan
+import org.apache.spark.sql.hive.HiveExternalCatalog
+import org.apache.spark.sql.internal.{SessionState, SharedState}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.parser.CarbonSparkSqlParserUtil.{checkIfDuplicateColumnExists, convertDbNameToLowerCase, validateStreamingProperty}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.streaming.Trigger
+import org.apache.spark.sql.types.{AbstractDataType, CharType, DataType, Metadata, StringType, StructField, VarcharType}
+import org.apache.spark.sql.util.SparkSQLUtil
+import org.apache.spark.unsafe.types.UTF8String
+
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.SchemaReader
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+import org.apache.carbondata.mv.plans.modular.{GroupBy, ModularPlan, Select}
+import org.apache.carbondata.spark.CarbonOption
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+object CarbonToSparkAdapter {

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



[GitHub] [carbondata] Jeromestein edited a comment on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
Jeromestein edited a comment on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-888871930






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

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647102389



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
##########
@@ -72,7 +72,7 @@ abstract class CarbonRDD[T: ClassTag](
   def internalCompute(split: Partition, context: TaskContext): Iterator[T]
 
   final def compute(split: Partition, context: TaskContext): Iterator[T] = {
-    TaskContext.get.addTaskCompletionListener(_ => ThreadLocalSessionInfo.unsetAll())
+    SparkVersionAdapter.addTaskCompletionListener(ThreadLocalSessionInfo.unsetAll())

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-851853444


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647124186



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
##########
@@ -1551,26 +1552,28 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
             sql("insert into hive_table select 'a','b','1'");
         }
 
-        val e2 = intercept[SparkException] {
-            sql("insert into hive_table2 select 'a','b','binary'");
-        }
+        if (!SparkUtil.isSparkVersionXAndAbove("3")) {
+            val e2 = intercept[SparkException] {
+                sql("insert into hive_table2 select 'a','b','binary'");
+            }
 
-        assert(e2.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e2.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))
 
-        val eInt2 = intercept[Exception] {
-            sql("insert into hive_table2 select 'a','b','1'");
-        }
+            val eInt2 = intercept[Exception] {
+                sql("insert into hive_table2 select 'a','b','1'");
+            }
 
-        val e3 = intercept[SparkException] {
-            sql("insert into parquet_table select 'a','b','binary'");
-        }
+            val e3 = intercept[SparkException] {
+                sql("insert into parquet_table select 'a','b','binary'");
+            }
 
-        assert(e3.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e3.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))

Review comment:
       if condition has been added above, needed for that




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647101776



##########
File path: mv/plan/src/main/spark3.1/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.mv.plans.modular
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeSeq, Expression, ExprId, NamedExpression, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, AggregateMode}
+import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, CollapseProject, CollapseRepartition, CollapseWindow, ColumnPruning, CombineFilters, CombineUnions, ConstantFolding, EliminateLimits, EliminateOuterJoin, EliminateSerialization, EliminateSorts, FoldablePropagation, NullPropagation, PushDownPredicates, PushPredicateThroughJoin, PushProjectionThroughUnion, RemoveDispensableExpressions, RemoveRedundantAliases, ReorderAssociativeOperator, ReorderJoin, RewriteCorrelatedScalarSubquery, SimplifyBinaryComparison, SimplifyCaseConversionExpressions, SimplifyCasts, SimplifyConditionals}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Join, LogicalPlan, Statistics, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+import org.apache.carbondata.mv.plans.util.BirdcageOptimizer
+
+object SparkVersionHelper {
+  def getStatisticsObj(outputList: Seq[NamedExpression],
+      plan: LogicalPlan, stats: Statistics,
+      aliasMap: Option[AttributeMap[Attribute]] = None): Statistics = {
+    val output = outputList.map(_.toAttribute)
+    val mapSeq = plan.collect { case n: logical.LeafNode => n }.map {
+      table => AttributeMap(table.output.zip(output))
+    }
+    val rewrites = mapSeq.head
+    val attributes: AttributeMap[ColumnStat] = stats.attributeStats
+    var attributeStats = AttributeMap(attributes.iterator
+      .map { pair => (rewrites(pair._1), pair._2) }.toSeq)
+    if (aliasMap.isDefined) {
+      attributeStats = AttributeMap(
+        attributeStats.map(pair => (aliasMap.get(pair._1), pair._2)).toSeq)
+    }
+    Statistics(stats.sizeInBytes, stats.rowCount, attributeStats)
+  }
+
+  def getOptimizedPlan(s: SubqueryExpression): LogicalPlan = {
+    val Subquery(newPlan, _) = BirdcageOptimizer.execute(Subquery.fromExpression(s))
+    newPlan
+  }
+
+  def normalizeExpressions[T <: Expression](r: T, attrs: AttributeSeq): T = {
+    QueryPlan.normalizeExpressions(r, attrs)
+  }
+
+  def attributeMap(rAliasMap: AttributeMap[Attribute]) : AttributeMap[Alias] = {
+    rAliasMap.asInstanceOf[AttributeMap[Alias]]
+  }
+
+  def seqOfRules : Seq[Rule[LogicalPlan]] = {
+    Seq(
+      // Operator push down
+      PushProjectionThroughUnion,
+      ReorderJoin,
+      EliminateOuterJoin,
+      PushPredicateThroughJoin,
+      PushDownPredicates,
+      ColumnPruning,
+      // Operator combine
+      CollapseRepartition,
+      CollapseProject,
+      CollapseWindow,
+      CombineFilters,
+      EliminateLimits,
+      CombineUnions,
+      // Constant folding and strength reduction
+      NullPropagation,
+      FoldablePropagation,
+      ConstantFolding,
+      ReorderAssociativeOperator,
+      // No need to apply LikeSimplification rule while creating MV
+      // as modular plan asCompactSql will be set in schema
+      //        LikeSimplification,
+      BooleanSimplification,
+      SimplifyConditionals,
+      RemoveDispensableExpressions,
+      SimplifyBinaryComparison,
+      EliminateSorts,
+      SimplifyCasts,
+      SimplifyCaseConversionExpressions,
+      RewriteCorrelatedScalarSubquery,
+      EliminateSerialization,
+      RemoveRedundantAliases)
+  }
+
+}
+
+trait getVerboseString extends LeafNode {
+  def verboseString: String = toString
+}
+
+trait groupByUnaryNode extends UnaryNode {
+  override def verboseString(maxFields: Int): String = super.verboseString(maxFields)
+
+  override def mapProductIterator[B](f: Any => B)(implicit evidence$1: ClassTag[B]): Array[B] = {
+    super.mapProductIterator(f)
+  }
+
+  override def mapChildren(f: ModularPlan => ModularPlan) : ModularPlan = {
+    val groupBy = super.mapChildren(f)
+    if (this.rewritten && !groupBy.rewritten) {
+      groupBy.setRewritten()
+    }
+    groupBy
+  }
+}
+
+trait selectModularPlan extends ModularPlan {

Review comment:
       done

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -232,9 +236,9 @@ object CarbonToSparkAdapter {
 }
 
 class CarbonOptimizer(
-    session: SparkSession,
-    catalog: SessionCatalog,
-    optimizer: Optimizer) extends Optimizer(catalog) {
+                       session: SparkSession,

Review comment:
       done

##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -187,11 +193,10 @@ object CarbonToSparkAdapter {
   }
 }
 
-
 class CarbonOptimizer(
-    session: SparkSession,
-    catalog: SessionCatalog,
-    optimizer: Optimizer) extends Optimizer(catalog) {
+                       session: SparkSession,

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644537339



##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/StructuredStreamingExample.scala
##########
@@ -151,7 +151,7 @@ object StructuredStreamingExample {
           // Write data from socket stream to carbondata file
           qry = readSocketDF.writeStream
             .format("carbondata")
-            .trigger(ProcessingTime("5 seconds"))
+            .trigger(CarbonToSparkAdapter.getProcessingTime("5 seconds"))

Review comment:
       added to scala package from spark_version




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

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646327536



##########
File path: mv/plan/src/main/spark2.4/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala
##########
@@ -17,9 +17,11 @@
 
 package org.apache.carbondata.mv.plans.modular
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, ExprId, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSeq, Expression, ExprId, NamedExpression, SubqueryExpression}

Review comment:
       revert this




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655296422



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
##########
@@ -65,7 +65,10 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
   protected val ESCAPECHAR = carbonKeyWord("ESCAPECHAR")
   protected val EXCLUDE = carbonKeyWord("EXCLUDE")
   protected val EXPLAIN = carbonKeyWord("EXPLAIN")
-  protected val EXTENDED = carbonKeyWord("EXTENDED")
+  protected val MODE = carbonKeyWord("EXTENDED") |
+                       carbonKeyWord("CODEGEN") |
+                       carbonKeyWord("COST") |
+                       carbonKeyWord("FORMATTED")

Review comment:
       JIRA: https://issues.apache.org/jira/browse/CARBONDATA-4201
   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:
users@infra.apache.org



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r645397625



##########
File path: index/examples/pom.xml
##########
@@ -81,9 +81,6 @@
   <profiles>
     <profile>
       <id>spark-2.3</id>
-      <activation>
-        <activeByDefault>true</activeByDefault>
-      </activation>

Review comment:
       no new profile is added 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:
users@infra.apache.org



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655893501



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSet, Expression, ExprId, NamedExpression, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FilePartition, PartitionedFile}
+import org.apache.spark.sql.hive.HiveExternalCatalog
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.types.{DataType, Metadata, StringType}
+
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+
+object CarbonToSparkAdapter extends SparkVersionAdapter {
+
+  def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]): FilePartition = {
+    FilePartition(index, files.toArray)
+  }
+
+  def addSparkSessionListener(sparkSession: SparkSession): Unit = {
+    sparkSession.sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        CarbonEnv.carbonEnvMap.remove(sparkSession)
+        ThreadLocalSessionInfo.unsetAll()
+      }
+    })
+  }
+
+  def addSparkListener(sparkContext: SparkContext): Unit = {
+    sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        SparkSession.setDefaultSession(null)
+      }
+    })
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Option[String],
+      attrRef : NamedExpression = null): AttributeReference = {
+    val qf = if (qualifier.nonEmpty) Seq(qualifier.get) else Seq.empty
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qf)
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Seq[String]): AttributeReference = {
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qualifier)
+  }
+
+  def lowerCaseAttribute(expression: Expression): Expression = expression.transform {
+    case attr: AttributeReference =>
+      CarbonToSparkAdapter.createAttributeReference(
+        attr.name.toLowerCase,
+        attr.dataType,
+        attr.nullable,
+        attr.metadata,
+        attr.exprId,
+        attr.qualifier)
+  }
+
+  def createAttributeReference(attr: AttributeReference,
+      attrName: String,
+      newSubsume: String): AttributeReference = {
+    AttributeReference(attrName, attr.dataType)(
+      exprId = attr.exprId,
+      qualifier = newSubsume.split("\n").map(_.trim))
+  }
+
+  def createScalaUDF(s: ScalaUDF, reference: AttributeReference): ScalaUDF = {
+    s.copy(children = Seq(reference))
+  }
+
+  def createExprCode(code: String, isNull: String, value: String, dataType: DataType): ExprCode = {
+    ExprCode(
+      code"$code",
+      JavaCode.isNullVariable(isNull),
+      JavaCode.variable(value, dataType))
+  }
+
+  def createAliasRef(
+      child: Expression,
+      name: String,
+      exprId: ExprId = NamedExpression.newExprId,
+      qualifier: Seq[String] = Seq.empty,
+      explicitMetadata: Option[Metadata] = None) : Alias = {
+    Alias(child, name)(exprId, qualifier, explicitMetadata)
+  }
+
+  def createAliasRef(
+      child: Expression,
+      name: String,
+      exprId: ExprId,
+      qualifier: Option[String]) : Alias = {
+    Alias(child, name)(exprId,
+      if (qualifier.isEmpty) Seq.empty else Seq(qualifier.get),
+      None)

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644538246



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
##########
@@ -183,7 +183,7 @@ object DataLoadProcessorStepOnSpark {
     val rowConverter = new RowConverterImpl(conf.getDataFields, conf, badRecordLogger)
     rowConverter.initialize()
 
-    TaskContext.get().addTaskCompletionListener { context =>
+    CarbonToSparkAdapter.addTaskCompletionListener {

Review comment:
       addTaskCompletionListener API change in Spark3.1




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-848689465


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


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

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



[GitHub] [carbondata] jbonofre commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
jbonofre commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-888794387


   @Jeromestein Hi, thanks for your interest in CarbonData. You can be in touch via mailing list.


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

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

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



[GitHub] [carbondata] kunal642 removed a comment on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 removed a comment on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-855624385


   LGTM
   
   @vikramahuja1001 please handle the comments and/or reply to the comments.. lets get this merged ASAP


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-865696051






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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655281093



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/TestIndexCommand.scala
##########
@@ -39,7 +39,9 @@ class TestIndexCommand extends QueryTest with BeforeAndAfterAll {
 
   val newClass = "org.apache.spark.sql.CarbonSource"
 
-  test("test index create: don't support using non-exist class") {
+  // TODO: This issue will be fixed during alter change column support for spark 3.1.1
+  // REFER: https://issues.apache.org/jira/browse/CARBONDATA-4210
+  ignore("test index create: don't support using non-exist class") {

Review comment:
       changed these test cases back, and changed the expected error message

##########
File path: integration/spark/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
##########
@@ -38,13 +38,13 @@ import org.apache.carbondata.spark.exception.ProcessMetaDataException
 
 class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll {
 
-  var threshold: Int = _
+  var threshold: String = _
 
   override def beforeAll {
 
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
-    threshold = sqlContext.getConf("spark.sql.autoBroadcastJoinThreshold").toInt
+    threshold = sqlContext.getConf("spark.sql.autoBroadcastJoinThreshold")

Review comment:
       changed

##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneIndexExample.scala
##########
@@ -61,37 +61,37 @@ object LuceneIndexExample {
       """.stripMargin)
 
     // 1. Compare the performance:
-
-    def time(code: => Unit): Double = {
-      val start = System.currentTimeMillis()
-      code
-      // return time in second
-      (System.currentTimeMillis() - start).toDouble / 1000
-    }
-
-    val timeWithoutLuceneIndex = time {
-
-      spark.sql(
-        s"""
-           | SELECT count(*)
-           | FROM personTable where id like '% test1 %'
-      """.stripMargin).show()
-
-    }
-
-    val timeWithLuceneIndex = time {
-
-      spark.sql(
-        s"""
-           | SELECT count(*)
-           | FROM personTable where TEXT_MATCH('id:test1')
-      """.stripMargin).show()
-
-    }
+    // TODO: Revert this after SPARK 3.1 fix

Review comment:
       changed this

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/CsvRDDHelper.scala
##########
@@ -26,7 +26,7 @@ import org.apache.hadoop.mapreduce.{TaskAttemptID, TaskType}
 import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.{CarbonToSparkAdapter, SparkSession}

Review comment:
       That old package has been renamed

##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneIndexExample.scala
##########
@@ -60,40 +60,6 @@ object LuceneIndexExample {
          | AS 'lucene'
       """.stripMargin)
 
-    // 1. Compare the performance:
-
-    def time(code: => Unit): Double = {
-      val start = System.currentTimeMillis()
-      code
-      // return time in second
-      (System.currentTimeMillis() - start).toDouble / 1000
-    }
-
-    val timeWithoutLuceneIndex = time {
-
-      spark.sql(
-        s"""
-           | SELECT count(*)
-           | FROM personTable where id like '% test1 %'
-      """.stripMargin).show()
-
-    }
-
-    val timeWithLuceneIndex = time {
-
-      spark.sql(

Review comment:
       done

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,735 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener, SessionCatalog}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, ExprId, Expression, ExpressionSet, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{JoinType, QueryPlan, logical}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode}
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, SQLExecution, ShuffledRowRDD, SimpleMode, SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, RefreshTableCommand, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FilePartition, PartitionedFile}
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.metric.SQLShuffleWriteMetricsReporter
+import org.apache.spark.sql.execution.strategy.CarbonDataSourceScan
+import org.apache.spark.sql.hive.HiveExternalCatalog
+import org.apache.spark.sql.internal.{SessionState, SharedState}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.parser.CarbonSparkSqlParserUtil.{checkIfDuplicateColumnExists, convertDbNameToLowerCase, validateStreamingProperty}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.streaming.Trigger
+import org.apache.spark.sql.types.{AbstractDataType, CharType, DataType, Metadata, StringType, StructField, VarcharType}
+import org.apache.spark.sql.util.SparkSQLUtil
+import org.apache.spark.unsafe.types.UTF8String
+
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.SchemaReader
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+import org.apache.carbondata.mv.plans.modular.{GroupBy, ModularPlan, Select}
+import org.apache.carbondata.spark.CarbonOption
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+object CarbonToSparkAdapter {

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
##########
@@ -65,7 +65,10 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
   protected val ESCAPECHAR = carbonKeyWord("ESCAPECHAR")
   protected val EXCLUDE = carbonKeyWord("EXCLUDE")
   protected val EXPLAIN = carbonKeyWord("EXPLAIN")
-  protected val EXTENDED = carbonKeyWord("EXTENDED")
+  protected val MODE = carbonKeyWord("EXTENDED") |
+                       carbonKeyWord("CODEGEN") |
+                       carbonKeyWord("COST") |
+                       carbonKeyWord("FORMATTED")

Review comment:
       JIRA: https://issues.apache.org/jira/browse/CARBONDATA-4201
   Done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/joins/BroadCastPolygonFilterPushJoin.scala
##########
@@ -47,16 +45,13 @@ case class BroadCastPolygonFilterPushJoin(
     leftKeys: Seq[Expression],
     rightKeys: Seq[Expression],
     joinType: JoinType,
-    buildSide: BuildSide,
     condition: Option[Expression],
     left: SparkPlan,
     right: SparkPlan
-) extends BinaryExecNode with HashJoin {

Review comment:
       methods overridden from HashJoin needed to be removed, so this itself was not needed.

##########
File path: processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/RowParserImpl.java
##########
@@ -109,7 +109,9 @@ public RowParserImpl(DataField[] output, CarbonDataLoadConfiguration configurati
       return new String[numberOfColumns];
     }
     // If number of columns are less in a row then create new array with same size of header.

Review comment:
       done

##########
File path: integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/CarbonDataSourceScanHelper.scala
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.spark.sql
+
+import org.apache.spark.CarbonInputMetrics
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression => SparkExpression}
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.execution.{ColumnarBatchScan, DataSourceScanExec}
+import org.apache.spark.sql.execution.strategy.CarbonPlanHelper
+import org.apache.spark.sql.optimizer.CarbonFilters
+
+import org.apache.carbondata.core.index.IndexFilter
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.scan.expression.Expression
+import org.apache.carbondata.core.scan.expression.logical.AndExpression
+import org.apache.carbondata.hadoop.CarbonProjection
+import org.apache.carbondata.spark.rdd.CarbonScanRDD
+
+abstract class  CarbonDataSourceScanHelper(relation: CarbonDatasourceHadoopRelation,
+                                           output: Seq[Attribute],
+    partitionFilters: Seq[SparkExpression],
+    pushedDownFilters: Seq[Expression],
+    pushedDownProjection: CarbonProjection,
+    directScanSupport: Boolean,
+    extraRDD: Option[(RDD[InternalRow], Boolean)],
+    segmentIds: Option[String])
+  extends DataSourceScanExec with ColumnarBatchScan {

Review comment:
       done

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
##########
@@ -1551,26 +1552,28 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
             sql("insert into hive_table select 'a','b','1'");
         }
 
-        val e2 = intercept[SparkException] {
-            sql("insert into hive_table2 select 'a','b','binary'");
-        }
+        if (!SparkUtil.isSparkVersionXAndAbove("3")) {
+            val e2 = intercept[SparkException] {
+                sql("insert into hive_table2 select 'a','b','binary'");
+            }
 
-        assert(e2.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e2.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))
 
-        val eInt2 = intercept[Exception] {
-            sql("insert into hive_table2 select 'a','b','1'");
-        }
+            val eInt2 = intercept[Exception] {
+                sql("insert into hive_table2 select 'a','b','1'");
+            }
 
-        val e3 = intercept[SparkException] {
-            sql("insert into parquet_table select 'a','b','binary'");
-        }
+            val e3 = intercept[SparkException] {
+                sql("insert into parquet_table select 'a','b','binary'");
+            }
 
-        assert(e3.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e3.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))
 
-        val eInt3 = intercept[Exception] {
-            sql("insert into parquet_table select 'a','b','1'");
+            val eInt3 = intercept[Exception] {
+                sql("insert into parquet_table select 'a','b','1'");
+            }

Review comment:
       it is needed or else spark will not be able to find the path to the carbon file

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/merge/MergeTestCase.scala
##########
@@ -842,7 +842,7 @@ class MergeTestCase extends QueryTest with BeforeAndAfterAll {
       Row("d", "3")
     ).asJava, StructType(Seq(StructField("key", StringType), StructField("value", StringType))))
 
-    initframe.write
+    initframe.repartition(1).write

Review comment:
       it is needed or else spark will not be able to find the path to the carbon file

##########
File path: mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/Printers.scala
##########
@@ -358,6 +359,12 @@ trait Printers {
           ""
         }
         qualifierPrefix + quoteIdentifier(child.name) + " AS " + quoteIdentifier(a.name)
+      case a@Alias(child: AggregateExpression, _) =>
+        child.sql + " AS " + quoteIdentifier(a.name)

Review comment:
       done

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
##########
@@ -1551,26 +1552,28 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
             sql("insert into hive_table select 'a','b','1'");
         }
 
-        val e2 = intercept[SparkException] {
-            sql("insert into hive_table2 select 'a','b','binary'");
-        }
+        if (!SparkUtil.isSparkVersionXAndAbove("3")) {
+            val e2 = intercept[SparkException] {
+                sql("insert into hive_table2 select 'a','b','binary'");
+            }
 
-        assert(e2.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e2.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))
 
-        val eInt2 = intercept[Exception] {
-            sql("insert into hive_table2 select 'a','b','1'");
-        }
+            val eInt2 = intercept[Exception] {
+                sql("insert into hive_table2 select 'a','b','1'");
+            }
 
-        val e3 = intercept[SparkException] {
-            sql("insert into parquet_table select 'a','b','binary'");
-        }
+            val e3 = intercept[SparkException] {
+                sql("insert into parquet_table select 'a','b','binary'");
+            }
 
-        assert(e3.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e3.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))
 
-        val eInt3 = intercept[Exception] {
-            sql("insert into parquet_table select 'a','b','1'");
+            val eInt3 = intercept[Exception] {
+                sql("insert into parquet_table select 'a','b','1'");
+            }

Review comment:
       if condition has been added, needed for that

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CarbonShowSegmentsAsSelectCommand.scala
##########
@@ -46,12 +49,25 @@ case class CarbonShowSegmentsAsSelectCommand(
     Checker.validateTableExists(databaseNameOp, tableName, sparkSession)
     CarbonEnv.getCarbonTable(databaseNameOp, tableName)(sparkSession)
   }
-  private lazy val df = createDataFrame
+
+  val tempViewName: String = makeTempViewName(carbonTable)
+
+  val df: DataFrame = createDataFrame
 
   override def output: Seq[Attribute] = {
-    df.queryExecution.analyzed.output.map { attr =>
-      AttributeReference(attr.name, attr.dataType, nullable = true)()
+    var attrList: Seq[Attribute] = Seq()

Review comment:
       done

##########
File path: examples/spark/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
##########
@@ -104,7 +104,8 @@ class RunExamples extends QueryTest with BeforeAndAfterAll {
     TableLevelCompactionOptionExample.exampleBody(spark)
   }
 
-  test("LuceneIndexExample") {
+  // Below test case ignored due to the Deadlock in spark code

Review comment:
       Not reported as it is very specific to Carbondata and this issue is not reproduced in native spark code without carbondata.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/IUDCommonUtil.scala
##########
@@ -44,7 +44,7 @@ object IUDCommonUtil {
       case unresolvedRelation: UnresolvedRelation =>
         val dbAndTb =
           sparkSession.sessionState.catalog.asInstanceOf[HiveSessionCatalog].getCurrentDatabase +
-          "." + unresolvedRelation.tableIdentifier.table
+          "." + CarbonToSparkAdapter.getTableIdentifier(unresolvedRelation).get.table

Review comment:
       Done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
##########
@@ -135,34 +137,34 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
       var transformed: Boolean = false
       // Create a dummy projection to include filter conditions
       var newPlan: LogicalPlan = null
-      if (table.tableIdentifier.database.isDefined) {
+      if (tableIdentifier.database.isDefined) {
         newPlan = parser.parsePlan("select * from  " +
-           table.tableIdentifier.database.getOrElse("") + "." +
-           table.tableIdentifier.table + " " + alias.getOrElse("") + " " + filter)
+           tableIdentifier.database.getOrElse("") + "." +

Review comment:
       Done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
##########
@@ -772,58 +774,62 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     }
 
   def getFields(schema: Seq[StructField], isExternal: Boolean = false): Seq[Field] = {
-    def getScannerInput(col: StructField,
-        columnComment: String,
-        columnName: String) = {
-      if (col.dataType.catalogString == "float" && !isExternal) {
-        '`' + columnName + '`' + " double" + columnComment
+    schema.map { col =>
+      // TODO: Spark has started supporting CharType/VarChar types in Spark 3.1 but both are
+      //  marked as experimental. Adding a hack to change to string for now.

Review comment:
       Done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
##########
@@ -835,10 +841,21 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         field.dataType = Some("double")
       }
       field.rawSchema = scannerInput
-      if (col.getComment().isDefined) {
+      if (comment.isDefined) {
         field.columnComment = plainComment
       }
       field
+  }
+
+  def getScannerInput(dataType: DataType,
+                      columnComment: String,
+                      columnName: String,
+                      isExternal: Boolean): String = {

Review comment:
       done

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSet, Expression, ExprId, NamedExpression, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FilePartition, PartitionedFile}
+import org.apache.spark.sql.hive.HiveExternalCatalog
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.types.{DataType, Metadata, StringType}
+
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+
+object CarbonToSparkAdapter extends SparkVersionAdapter {
+
+  def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]): FilePartition = {
+    FilePartition(index, files.toArray)
+  }
+
+  def addSparkSessionListener(sparkSession: SparkSession): Unit = {
+    sparkSession.sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        CarbonEnv.carbonEnvMap.remove(sparkSession)
+        ThreadLocalSessionInfo.unsetAll()
+      }
+    })
+  }
+
+  def addSparkListener(sparkContext: SparkContext): Unit = {
+    sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        SparkSession.setDefaultSession(null)
+      }
+    })
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Option[String],
+      attrRef : NamedExpression = null): AttributeReference = {
+    val qf = if (qualifier.nonEmpty) Seq(qualifier.get) else Seq.empty
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qf)
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Seq[String]): AttributeReference = {
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qualifier)
+  }
+
+  def lowerCaseAttribute(expression: Expression): Expression = expression.transform {
+    case attr: AttributeReference =>
+      CarbonToSparkAdapter.createAttributeReference(
+        attr.name.toLowerCase,
+        attr.dataType,
+        attr.nullable,
+        attr.metadata,
+        attr.exprId,
+        attr.qualifier)
+  }
+
+  def createAttributeReference(attr: AttributeReference,
+      attrName: String,
+      newSubsume: String): AttributeReference = {
+    AttributeReference(attrName, attr.dataType)(
+      exprId = attr.exprId,
+      qualifier = newSubsume.split("\n").map(_.trim))
+  }
+
+  def createScalaUDF(s: ScalaUDF, reference: AttributeReference): ScalaUDF = {
+    s.copy(children = Seq(reference))
+  }
+
+  def createExprCode(code: String, isNull: String, value: String, dataType: DataType): ExprCode = {
+    ExprCode(
+      code"$code",
+      JavaCode.isNullVariable(isNull),
+      JavaCode.variable(value, dataType))
+  }
+
+  def createAliasRef(
+      child: Expression,
+      name: String,
+      exprId: ExprId = NamedExpression.newExprId,
+      qualifier: Seq[String] = Seq.empty,
+      explicitMetadata: Option[Metadata] = None) : Alias = {
+    Alias(child, name)(exprId, qualifier, explicitMetadata)
+  }
+
+  def createAliasRef(
+      child: Expression,
+      name: String,
+      exprId: ExprId,
+      qualifier: Option[String]) : Alias = {
+    Alias(child, name)(exprId,
+      if (qualifier.isEmpty) Seq.empty else Seq(qualifier.get),
+      None)

Review comment:
       done

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.spark.sql.parser
+
+import scala.collection.mutable
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.sql.{CarbonThreadUtil, CarbonToSparkAdapter, SparkSession}
+import org.apache.spark.sql.catalyst.parser.{AbstractSqlParser, SqlBaseParser}
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.{SparkSqlAstBuilder, SparkSqlParser}
+import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
+import org.apache.spark.sql.types.StructField
+import org.apache.spark.sql.util.CarbonException
+import org.apache.spark.util.CarbonReflectionUtils
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * Concrete parser for Spark SQL statements and carbon specific
+ * statements
+ */
+class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends SparkSqlParser {
+
+  val parser = new CarbonSpark2SqlParser
+
+  override val astBuilder = CarbonReflectionUtils.getAstBuilder(conf, parser, sparkSession)
+
+  private val substitutor = new VariableSubstitution
+
+  override def parsePlan(sqlText: String): LogicalPlan = {
+    CarbonThreadUtil.updateSessionInfoToCurrentThread(sparkSession)
+    try {
+      val parsedPlan = super.parsePlan(sqlText)
+      CarbonScalaUtil.cleanParserThreadLocals
+      parsedPlan
+    } catch {
+      case ce: MalformedCarbonCommandException =>
+        CarbonScalaUtil.cleanParserThreadLocals
+        throw ce
+      case ex: Throwable =>
+        try {
+          parser.parse(sqlText)
+        } catch {
+          case mce: MalformedCarbonCommandException =>
+            throw mce
+          case e: Throwable =>
+            CarbonException.analysisException(
+              s"""== Parse1 ==
+                 |${ex.getMessage}
+                 |== Parse2 ==
+                 |${e.getMessage}
+               """.stripMargin.trim)
+        }
+    }
+  }
+
+  protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = {
+    super.parse(substitutor.substitute(command))(toResult)
+  }
+}
+
+class CarbonHelperSqlAstBuilder(conf: SQLConf,
+    parser: CarbonSpark2SqlParser,
+    sparkSession: SparkSession)
+  extends SparkSqlAstBuilderWrapper(conf) {
+  /**
+   * Parse a key-value map from a [[TablePropertyListContext]], assuming all values are specified.
+   */
+  override def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = {
+    val props = visitTablePropertyList(ctx)
+    CarbonSparkSqlParserUtil.visitPropertyKeyValues(ctx, props)
+  }
+
+  def getPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String]
+  = {

Review comment:
       done

##########
File path: mv/plan/src/main/spark2.3/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala
##########
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.mv.plans.modular
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, ExprId, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, ExprId, NamedExpression}

Review comment:
       Checkstyle was not running on this class initially, if changed back then checkstyle will fail.




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

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646319099



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
##########
@@ -65,7 +65,10 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
   protected val ESCAPECHAR = carbonKeyWord("ESCAPECHAR")
   protected val EXCLUDE = carbonKeyWord("EXCLUDE")
   protected val EXPLAIN = carbonKeyWord("EXPLAIN")
-  protected val EXTENDED = carbonKeyWord("EXTENDED")
+  protected val MODE = carbonKeyWord("EXTENDED") |
+                       carbonKeyWord("CODEGEN") |
+                       carbonKeyWord("COST") |
+                       carbonKeyWord("FORMATTED")

Review comment:
       create JIRA for other MODES and add a comment 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:
users@infra.apache.org



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644556501



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,735 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener, SessionCatalog}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, ExprId, Expression, ExpressionSet, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{JoinType, QueryPlan, logical}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode}
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, SQLExecution, ShuffledRowRDD, SimpleMode, SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, RefreshTableCommand, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FilePartition, PartitionedFile}
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.metric.SQLShuffleWriteMetricsReporter
+import org.apache.spark.sql.execution.strategy.CarbonDataSourceScan
+import org.apache.spark.sql.hive.HiveExternalCatalog
+import org.apache.spark.sql.internal.{SessionState, SharedState}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.parser.CarbonSparkSqlParserUtil.{checkIfDuplicateColumnExists, convertDbNameToLowerCase, validateStreamingProperty}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.streaming.Trigger
+import org.apache.spark.sql.types.{AbstractDataType, CharType, DataType, Metadata, StringType, StructField, VarcharType}
+import org.apache.spark.sql.util.SparkSQLUtil
+import org.apache.spark.unsafe.types.UTF8String
+
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.SchemaReader
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+import org.apache.carbondata.mv.plans.modular.{GroupBy, ModularPlan, Select}
+import org.apache.carbondata.spark.CarbonOption
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+object CarbonToSparkAdapter {
+
+  def addSparkSessionListener(sparkSession: SparkSession): Unit = {
+    sparkSession.sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        CarbonEnv.carbonEnvMap.remove(sparkSession)
+        ThreadLocalSessionInfo.unsetAll()
+      }
+    })
+  }
+
+  def addSparkListener(sparkContext: SparkContext): Unit = {
+    sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        SparkSession.setDefaultSession(null)
+      }
+    })
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Option[String],
+      attrRef : NamedExpression = null): AttributeReference = {
+    val qf = if (qualifier.nonEmpty) Seq(qualifier.get) else Seq.empty
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qf)
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Seq[String]): AttributeReference = {
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qualifier)
+  }
+
+  def lowerCaseAttribute(expression: Expression): Expression = expression.transform {
+    case attr: AttributeReference =>
+      CarbonToSparkAdapter.createAttributeReference(
+        attr.name.toLowerCase,
+        attr.dataType,
+        attr.nullable,
+        attr.metadata,
+        attr.exprId,
+        attr.qualifier)
+  }
+
+  def createAttributeReference(attr: AttributeReference,
+      attrName: String,
+      newSubsume: String): AttributeReference = {
+    AttributeReference(attrName, attr.dataType)(
+      exprId = attr.exprId,
+      qualifier = newSubsume.split("\n").map(_.trim))
+  }
+
+  def createScalaUDF(s: ScalaUDF, reference: AttributeReference): ScalaUDF = {
+    s.copy(children = Seq(reference))
+  }
+
+  def createExprCode(code: String, isNull: String, value: String, dataType: DataType): ExprCode = {
+    ExprCode(
+      code"$code",
+      JavaCode.isNullVariable(isNull),
+      JavaCode.variable(value, dataType))
+  }
+
+  def createAliasRef(
+      child: Expression,
+      name: String,
+      exprId: ExprId = NamedExpression.newExprId,
+      qualifier: Seq[String] = Seq.empty,
+      explicitMetadata: Option[Metadata] = None,
+      namedExpr: Option[NamedExpression] = None) : Alias = {

Review comment:
       done

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/avro/AvroFileFormatFactory.scala
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.spark.sql.avro

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655300176



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
##########
@@ -1551,26 +1552,28 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
             sql("insert into hive_table select 'a','b','1'");
         }
 
-        val e2 = intercept[SparkException] {
-            sql("insert into hive_table2 select 'a','b','binary'");
-        }
+        if (!SparkUtil.isSparkVersionXAndAbove("3")) {
+            val e2 = intercept[SparkException] {
+                sql("insert into hive_table2 select 'a','b','binary'");
+            }
 
-        assert(e2.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e2.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))
 
-        val eInt2 = intercept[Exception] {
-            sql("insert into hive_table2 select 'a','b','1'");
-        }
+            val eInt2 = intercept[Exception] {
+                sql("insert into hive_table2 select 'a','b','1'");
+            }
 
-        val e3 = intercept[SparkException] {
-            sql("insert into parquet_table select 'a','b','binary'");
-        }
+            val e3 = intercept[SparkException] {
+                sql("insert into parquet_table select 'a','b','binary'");
+            }
 
-        assert(e3.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e3.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))
 
-        val eInt3 = intercept[Exception] {
-            sql("insert into parquet_table select 'a','b','1'");
+            val eInt3 = intercept[Exception] {
+                sql("insert into parquet_table select 'a','b','1'");
+            }

Review comment:
       it is needed or else spark will not be able to find the path to the carbon file

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/merge/MergeTestCase.scala
##########
@@ -842,7 +842,7 @@ class MergeTestCase extends QueryTest with BeforeAndAfterAll {
       Row("d", "3")
     ).asJava, StructType(Seq(StructField("key", StringType), StructField("value", StringType))))
 
-    initframe.write
+    initframe.repartition(1).write

Review comment:
       it is needed or else spark will not be able to find the path to the carbon file




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

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



[GitHub] [carbondata] asfgit closed pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
asfgit closed pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141


   


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-848158231


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-848135381


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-865336191


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647119254



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkVersionAdapter.scala
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener, SessionCatalog}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, Expression, ExpressionSet, ExprId, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode}
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, ShuffledRowRDD, SimpleMode, SparkPlan, SQLExecution, UnaryExecNode}

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-851853858


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-848691115


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-854806997


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-851917782


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r645298630



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/util/CreateTableCommonUtil.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.spark.sql.util
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogUtils}
+import org.apache.spark.sql.execution.datasources.{DataSource, HadoopFsRelation}
+import org.apache.spark.sql.internal.SessionState
+import org.apache.spark.sql.sources.BaseRelation
+import org.apache.spark.sql.types.StructType
+
+object CreateTableCommonUtil {
+
+  def getNewTable(sparkSession: SparkSession, sessionState: SessionState,

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



[GitHub] [carbondata] vikramahuja1001 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-864942250






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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644524985



##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneIndexExample.scala
##########
@@ -30,7 +30,7 @@ object LuceneIndexExample {
 
   def main(args: Array[String]) {
     val spark = ExampleUtils.createSparkSession("LuceneIndexExample")
-    exampleBody(spark)
+      exampleBody(spark)

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



[GitHub] [carbondata] kunal642 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-866022360


   LGTM


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-864906851


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


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

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



[GitHub] [carbondata] Jeromestein commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
Jeromestein commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-888871930


   > @Jeromestein Hi, thanks for your interest in CarbonData. You can be in touch via mailing list.
   
   Thanks for your advice. 
   In fact, I've already used mailing list to post some thoughts and leave a message to other contributors, still few response though. 
   
   Now I am curious about the workflow of our community. What should I do before I decide to make a contribution next time?
   For example, I wanted to work on the time stamp issue (https://issues.apache.org/jira/browse/CARBONDATA-4247), and then found out that they had already pulled a request about it. (which cannot be found in mailing list...)
   
   


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

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

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



[GitHub] [carbondata] akashrn5 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655292078



##########
File path: examples/spark/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
##########
@@ -104,7 +104,8 @@ class RunExamples extends QueryTest with BeforeAndAfterAll {
     TableLevelCompactionOptionExample.exampleBody(spark)
   }
 
-  test("LuceneIndexExample") {
+  // Below test case ignored due to the Deadlock in spark code

Review comment:
       add a spark jira if its reported already, and add a TODO here to revert once the spark jira is resolved

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
##########
@@ -135,34 +137,34 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
       var transformed: Boolean = false
       // Create a dummy projection to include filter conditions
       var newPlan: LogicalPlan = null
-      if (table.tableIdentifier.database.isDefined) {
+      if (tableIdentifier.database.isDefined) {
         newPlan = parser.parsePlan("select * from  " +
-           table.tableIdentifier.database.getOrElse("") + "." +
-           table.tableIdentifier.table + " " + alias.getOrElse("") + " " + filter)
+           tableIdentifier.database.getOrElse("") + "." +

Review comment:
       can you replace with POINT in all places in this class?

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
##########
@@ -772,58 +774,62 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     }
 
   def getFields(schema: Seq[StructField], isExternal: Boolean = false): Seq[Field] = {
-    def getScannerInput(col: StructField,
-        columnComment: String,
-        columnName: String) = {
-      if (col.dataType.catalogString == "float" && !isExternal) {
-        '`' + columnName + '`' + " double" + columnComment
+    schema.map { col =>
+      // TODO: Spark has started supporting CharType/VarChar types in Spark 3.1 but both are
+      //  marked as experimental. Adding a hack to change to string for now.

Review comment:
       please mention the jira here in TODO

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/IUDCommonUtil.scala
##########
@@ -44,7 +44,7 @@ object IUDCommonUtil {
       case unresolvedRelation: UnresolvedRelation =>
         val dbAndTb =
           sparkSession.sessionState.catalog.asInstanceOf[HiveSessionCatalog].getCurrentDatabase +
-          "." + unresolvedRelation.tableIdentifier.table
+          "." + CarbonToSparkAdapter.getTableIdentifier(unresolvedRelation).get.table

Review comment:
       replace with constant POINT here

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.spark.sql.parser
+
+import scala.collection.mutable
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.sql.{CarbonThreadUtil, CarbonToSparkAdapter, SparkSession}
+import org.apache.spark.sql.catalyst.parser.{AbstractSqlParser, SqlBaseParser}
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.{SparkSqlAstBuilder, SparkSqlParser}
+import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
+import org.apache.spark.sql.types.StructField
+import org.apache.spark.sql.util.CarbonException
+import org.apache.spark.util.CarbonReflectionUtils
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * Concrete parser for Spark SQL statements and carbon specific
+ * statements
+ */
+class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends SparkSqlParser {
+
+  val parser = new CarbonSpark2SqlParser
+
+  override val astBuilder = CarbonReflectionUtils.getAstBuilder(conf, parser, sparkSession)
+
+  private val substitutor = new VariableSubstitution
+
+  override def parsePlan(sqlText: String): LogicalPlan = {
+    CarbonThreadUtil.updateSessionInfoToCurrentThread(sparkSession)
+    try {
+      val parsedPlan = super.parsePlan(sqlText)
+      CarbonScalaUtil.cleanParserThreadLocals
+      parsedPlan
+    } catch {
+      case ce: MalformedCarbonCommandException =>
+        CarbonScalaUtil.cleanParserThreadLocals
+        throw ce
+      case ex: Throwable =>
+        try {
+          parser.parse(sqlText)
+        } catch {
+          case mce: MalformedCarbonCommandException =>
+            throw mce
+          case e: Throwable =>
+            CarbonException.analysisException(
+              s"""== Parse1 ==
+                 |${ex.getMessage}
+                 |== Parse2 ==
+                 |${e.getMessage}
+               """.stripMargin.trim)
+        }
+    }
+  }
+
+  protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = {
+    super.parse(substitutor.substitute(command))(toResult)
+  }
+}
+
+class CarbonHelperSqlAstBuilder(conf: SQLConf,
+    parser: CarbonSpark2SqlParser,
+    sparkSession: SparkSession)
+  extends SparkSqlAstBuilderWrapper(conf) {
+  /**
+   * Parse a key-value map from a [[TablePropertyListContext]], assuming all values are specified.
+   */
+  override def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = {
+    val props = visitTablePropertyList(ctx)
+    CarbonSparkSqlParserUtil.visitPropertyKeyValues(ctx, props)
+  }
+
+  def getPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String]
+  = {

Review comment:
       move this line above

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSet, Expression, ExprId, NamedExpression, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FilePartition, PartitionedFile}
+import org.apache.spark.sql.hive.HiveExternalCatalog
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.types.{DataType, Metadata, StringType}
+
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+
+object CarbonToSparkAdapter extends SparkVersionAdapter {
+
+  def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]): FilePartition = {
+    FilePartition(index, files.toArray)
+  }
+
+  def addSparkSessionListener(sparkSession: SparkSession): Unit = {
+    sparkSession.sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        CarbonEnv.carbonEnvMap.remove(sparkSession)
+        ThreadLocalSessionInfo.unsetAll()
+      }
+    })
+  }
+
+  def addSparkListener(sparkContext: SparkContext): Unit = {
+    sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        SparkSession.setDefaultSession(null)
+      }
+    })
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Option[String],
+      attrRef : NamedExpression = null): AttributeReference = {
+    val qf = if (qualifier.nonEmpty) Seq(qualifier.get) else Seq.empty
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qf)
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Seq[String]): AttributeReference = {
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qualifier)
+  }
+
+  def lowerCaseAttribute(expression: Expression): Expression = expression.transform {
+    case attr: AttributeReference =>
+      CarbonToSparkAdapter.createAttributeReference(
+        attr.name.toLowerCase,
+        attr.dataType,
+        attr.nullable,
+        attr.metadata,
+        attr.exprId,
+        attr.qualifier)
+  }
+
+  def createAttributeReference(attr: AttributeReference,
+      attrName: String,
+      newSubsume: String): AttributeReference = {
+    AttributeReference(attrName, attr.dataType)(
+      exprId = attr.exprId,
+      qualifier = newSubsume.split("\n").map(_.trim))
+  }
+
+  def createScalaUDF(s: ScalaUDF, reference: AttributeReference): ScalaUDF = {
+    s.copy(children = Seq(reference))
+  }
+
+  def createExprCode(code: String, isNull: String, value: String, dataType: DataType): ExprCode = {
+    ExprCode(
+      code"$code",
+      JavaCode.isNullVariable(isNull),
+      JavaCode.variable(value, dataType))
+  }
+
+  def createAliasRef(
+      child: Expression,
+      name: String,
+      exprId: ExprId = NamedExpression.newExprId,
+      qualifier: Seq[String] = Seq.empty,
+      explicitMetadata: Option[Metadata] = None) : Alias = {
+    Alias(child, name)(exprId, qualifier, explicitMetadata)
+  }
+
+  def createAliasRef(
+      child: Expression,
+      name: String,
+      exprId: ExprId,
+      qualifier: Option[String]) : Alias = {
+    Alias(child, name)(exprId,
+      if (qualifier.isEmpty) Seq.empty else Seq(qualifier.get),
+      None)

Review comment:
       move this line above

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
##########
@@ -835,10 +841,21 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         field.dataType = Some("double")
       }
       field.rawSchema = scannerInput
-      if (col.getComment().isDefined) {
+      if (comment.isDefined) {
         field.columnComment = plainComment
       }
       field
+  }
+
+  def getScannerInput(dataType: DataType,
+                      columnComment: String,
+                      columnName: String,
+                      isExternal: Boolean): String = {

Review comment:
       please correct the code style here

##########
File path: mv/plan/src/main/spark2.3/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala
##########
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.mv.plans.modular
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, ExprId, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, ExprId, NamedExpression}

Review comment:
       revert this




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

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646328572



##########
File path: mv/plan/src/main/spark3.1/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.mv.plans.modular
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeSeq, Expression, ExprId, NamedExpression, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, AggregateMode}
+import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, CollapseProject, CollapseRepartition, CollapseWindow, ColumnPruning, CombineFilters, CombineUnions, ConstantFolding, EliminateLimits, EliminateOuterJoin, EliminateSerialization, EliminateSorts, FoldablePropagation, NullPropagation, PushDownPredicates, PushPredicateThroughJoin, PushProjectionThroughUnion, RemoveDispensableExpressions, RemoveRedundantAliases, ReorderAssociativeOperator, ReorderJoin, RewriteCorrelatedScalarSubquery, SimplifyBinaryComparison, SimplifyCaseConversionExpressions, SimplifyCasts, SimplifyConditionals}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Join, LogicalPlan, Statistics, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+import org.apache.carbondata.mv.plans.util.BirdcageOptimizer
+
+object SparkVersionHelper {
+  def getStatisticsObj(outputList: Seq[NamedExpression],
+      plan: LogicalPlan, stats: Statistics,
+      aliasMap: Option[AttributeMap[Attribute]] = None): Statistics = {
+    val output = outputList.map(_.toAttribute)
+    val mapSeq = plan.collect { case n: logical.LeafNode => n }.map {
+      table => AttributeMap(table.output.zip(output))
+    }
+    val rewrites = mapSeq.head
+    val attributes: AttributeMap[ColumnStat] = stats.attributeStats
+    var attributeStats = AttributeMap(attributes.iterator
+      .map { pair => (rewrites(pair._1), pair._2) }.toSeq)
+    if (aliasMap.isDefined) {
+      attributeStats = AttributeMap(
+        attributeStats.map(pair => (aliasMap.get(pair._1), pair._2)).toSeq)
+    }
+    Statistics(stats.sizeInBytes, stats.rowCount, attributeStats)
+  }
+
+  def getOptimizedPlan(s: SubqueryExpression): LogicalPlan = {
+    val Subquery(newPlan, _) = BirdcageOptimizer.execute(Subquery.fromExpression(s))
+    newPlan
+  }
+
+  def normalizeExpressions[T <: Expression](r: T, attrs: AttributeSeq): T = {
+    QueryPlan.normalizeExpressions(r, attrs)
+  }
+
+  def attributeMap(rAliasMap: AttributeMap[Attribute]) : AttributeMap[Alias] = {
+    rAliasMap.asInstanceOf[AttributeMap[Alias]]
+  }
+
+  def seqOfRules : Seq[Rule[LogicalPlan]] = {
+    Seq(
+      // Operator push down
+      PushProjectionThroughUnion,
+      ReorderJoin,
+      EliminateOuterJoin,
+      PushPredicateThroughJoin,
+      PushDownPredicates,
+      ColumnPruning,
+      // Operator combine
+      CollapseRepartition,
+      CollapseProject,
+      CollapseWindow,
+      CombineFilters,
+      EliminateLimits,
+      CombineUnions,
+      // Constant folding and strength reduction
+      NullPropagation,
+      FoldablePropagation,
+      ConstantFolding,
+      ReorderAssociativeOperator,
+      // No need to apply LikeSimplification rule while creating MV
+      // as modular plan asCompactSql will be set in schema
+      //        LikeSimplification,
+      BooleanSimplification,
+      SimplifyConditionals,
+      RemoveDispensableExpressions,
+      SimplifyBinaryComparison,
+      EliminateSorts,
+      SimplifyCasts,
+      SimplifyCaseConversionExpressions,
+      RewriteCorrelatedScalarSubquery,
+      EliminateSerialization,
+      RemoveRedundantAliases)
+  }
+
+}
+
+trait getVerboseString extends LeafNode {
+  def verboseString: String = toString
+}
+
+trait groupByUnaryNode extends UnaryNode {
+  override def verboseString(maxFields: Int): String = super.verboseString(maxFields)
+
+  override def mapProductIterator[B](f: Any => B)(implicit evidence$1: ClassTag[B]): Array[B] = {
+    super.mapProductIterator(f)
+  }
+
+  override def mapChildren(f: ModularPlan => ModularPlan) : ModularPlan = {
+    val groupBy = super.mapChildren(f)
+    if (this.rewritten && !groupBy.rewritten) {
+      groupBy.setRewritten()
+    }
+    groupBy
+  }
+}
+
+trait selectModularPlan extends ModularPlan {

Review comment:
       Change to "SelectModularPlan"




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

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



[GitHub] [carbondata] akashrn5 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646379260



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/rdd/CarbonRDD.scala
##########
@@ -72,7 +72,7 @@ abstract class CarbonRDD[T: ClassTag](
   def internalCompute(split: Partition, context: TaskContext): Iterator[T]
 
   final def compute(split: Partition, context: TaskContext): Iterator[T] = {
-    TaskContext.get.addTaskCompletionListener(_ => ThreadLocalSessionInfo.unsetAll())
+    SparkVersionAdapter.addTaskCompletionListener(ThreadLocalSessionInfo.unsetAll())

Review comment:
       please ad the scala bug in the comment of scala method `SparkVersionAdapter`

##########
File path: integration/spark/src/main/common2.4and3.1/org/apache/spark/sql/CarbonBoundReference.scala
##########
@@ -42,4 +42,4 @@ case class CarbonBoundReference(colExp: ColumnExpression, dataType: DataType, nu
   override def qualifier: Seq[String] = null
 
   override def newInstance(): NamedExpression = throw new UnsupportedOperationException
-}
\ No newline at end of file
+}

Review comment:
       revert this

##########
File path: integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/SparkVersionAdapter.scala
##########
@@ -0,0 +1,440 @@
+/*
+ * 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.spark.sql
+
+import scala.collection.mutable
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeSeq, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.expressions.codegen.{ExprCode, GeneratePredicate}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoTable, Join, LogicalPlan, OneRowRelation, Statistics}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.{QueryExecution, ShuffledRowRDD, SparkPlan, SQLExecution, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, RefreshTable}
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide}
+import org.apache.spark.sql.execution.strategy.CarbonDataSourceScan
+import org.apache.spark.sql.internal.{SessionState, SharedState}
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.parser.CarbonSparkSqlParserUtil.{checkIfDuplicateColumnExists, convertDbNameToLowerCase, validateStreamingProperty}
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{DataType, StructField}
+import org.apache.spark.unsafe.types.UTF8String
+
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.SchemaReader
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.spark.CarbonOption
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+object SparkVersionAdapter {

Review comment:
       please select the code from 63 to last and correct the style, (Ctrl + Shift + Alt + L)

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.spark.sql.parser
+
+import scala.collection.mutable
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.sql.{CarbonThreadUtil, SparkSession, SparkVersionAdapter}
+import org.apache.spark.sql.catalyst.parser.{AbstractSqlParser, SqlBaseParser}
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.{SparkSqlAstBuilder, SparkSqlParser}
+import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
+import org.apache.spark.sql.types.StructField
+import org.apache.spark.sql.util.CarbonException
+import org.apache.spark.util.CarbonReflectionUtils
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * Concrete parser for Spark SQL statements and carbon specific
+ * statements
+ */
+class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends SparkSqlParser {
+
+  val parser = new CarbonSpark2SqlParser
+
+  override val astBuilder = CarbonReflectionUtils.getAstBuilder(conf, parser, sparkSession)
+
+  private val substitutor = new VariableSubstitution
+
+  override def parsePlan(sqlText: String): LogicalPlan = {
+    CarbonThreadUtil.updateSessionInfoToCurrentThread(sparkSession)
+    try {
+      val parsedPlan = super.parsePlan(sqlText)
+      CarbonScalaUtil.cleanParserThreadLocals
+      parsedPlan
+    } catch {
+      case ce: MalformedCarbonCommandException =>
+        CarbonScalaUtil.cleanParserThreadLocals
+        throw ce
+      case ex: Throwable =>
+        try {
+          parser.parse(sqlText)
+        } catch {
+          case mce: MalformedCarbonCommandException =>
+            throw mce
+          case e: Throwable =>
+            CarbonException.analysisException(
+              s"""== Parse1 ==
+                 |${ex.getMessage}
+                 |== Parse2 ==
+                 |${e.getMessage}
+               """.stripMargin.trim)
+        }
+    }
+  }
+
+  protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = {
+    super.parse(substitutor.substitute(command))(toResult)
+  }
+}
+
+class CarbonHelperSqlAstBuilder(conf: SQLConf,
+    parser: CarbonSpark2SqlParser,
+    sparkSession: SparkSession)
+  extends SparkSqlAstBuilderWrapper(conf) {
+  /**
+   * Parse a key-value map from a [[TablePropertyListContext]], assuming all values are specified.
+   */
+  override def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = {
+    val props = visitTablePropertyList(ctx)
+    CarbonSparkSqlParserUtil.visitPropertyKeyValues(ctx, props)
+  }
+
+  def getPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String]
+  = {
+    Option(ctx).map(visitPropertyKeyValues)
+      .getOrElse(Map.empty)
+  }
+
+  def createCarbonTable(createTableTuple: (CreateTableHeaderContext, SkewSpecContext,
+    BucketSpecContext, PartitionFieldListContext, ColTypeListContext, TablePropertyListContext,
+    LocationSpecContext, Option[String], TerminalNode, QueryContext, String)): LogicalPlan = {
+    // val parser = new CarbonSpark2SqlParser

Review comment:
       remove this line

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkVersionAdapter.scala
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener, SessionCatalog}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, Expression, ExpressionSet, ExprId, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode}
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, ShuffledRowRDD, SimpleMode, SparkPlan, SQLExecution, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, RefreshTableCommand, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.metric.SQLShuffleWriteMetricsReporter
+import org.apache.spark.sql.execution.strategy.CarbonDataSourceScan
+import org.apache.spark.sql.internal.{SessionState, SharedState}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.parser.CarbonSparkSqlParserUtil.{checkIfDuplicateColumnExists, convertDbNameToLowerCase, validateStreamingProperty}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{AbstractDataType, CharType, DataType, Metadata, StringType, StructField, VarcharType}
+import org.apache.spark.sql.util.SparkSQLUtil
+import org.apache.spark.unsafe.types.UTF8String
+
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.SchemaReader
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+import org.apache.carbondata.spark.CarbonOption
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+object SparkVersionAdapter {
+
+  def getExplainCommandObj(logicalPlan: LogicalPlan = OneRowRelation(),
+                           mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(logicalPlan, ExplainMode.fromString(mode.getOrElse(SimpleMode.name)))
+  }
+
+  def getExplainCommandObj(mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(OneRowRelation(), ExplainMode.fromString(mode.getOrElse(SimpleMode.name)))
+  }
+
+  /**
+   * As a part of SPARK-24085 Hive tables supports scala subquery for
+   * the partitioned tables,so Carbon also needs to supports
+   * @param partitionSet
+   * @param filterPredicates
+   * @return
+   */
+  def getPartitionFilter(
+                          partitionSet: AttributeSet,
+                          filterPredicates: Seq[Expression]): Seq[Expression] = {

Review comment:
       correct the style here and check whole file once, select whole file and correct the style once

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
##########
@@ -1551,26 +1552,28 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
             sql("insert into hive_table select 'a','b','1'");
         }
 
-        val e2 = intercept[SparkException] {
-            sql("insert into hive_table2 select 'a','b','binary'");
-        }
+        if (!SparkUtil.isSparkVersionXAndAbove("3")) {
+            val e2 = intercept[SparkException] {
+                sql("insert into hive_table2 select 'a','b','binary'");
+            }
 
-        assert(e2.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e2.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))
 
-        val eInt2 = intercept[Exception] {
-            sql("insert into hive_table2 select 'a','b','1'");
-        }
+            val eInt2 = intercept[Exception] {
+                sql("insert into hive_table2 select 'a','b','1'");
+            }
 
-        val e3 = intercept[SparkException] {
-            sql("insert into parquet_table select 'a','b','binary'");
-        }
+            val e3 = intercept[SparkException] {
+                sql("insert into parquet_table select 'a','b','binary'");
+            }
 
-        assert(e3.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e3.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))
 
-        val eInt3 = intercept[Exception] {
-            sql("insert into parquet_table select 'a','b','1'");
+            val eInt3 = intercept[Exception] {
+                sql("insert into parquet_table select 'a','b','1'");
+            }

Review comment:
       revert

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
##########
@@ -169,7 +128,7 @@ case class CarbonDataSourceScan(
   override protected def doCanonicalize(): CarbonDataSourceScan = {
     CarbonDataSourceScan(
       relation,
-      output.map(QueryPlan.normalizeExprId(_, output)),
+      a,

Review comment:
       please rename to the proper variable name like `outputAttibutesAfterNormalizingExpressionIds`

##########
File path: mv/plan/src/main/spark3.1/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.mv.plans.modular
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeSeq, Expression, ExprId, NamedExpression, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, AggregateMode}
+import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, CollapseProject, CollapseRepartition, CollapseWindow, ColumnPruning, CombineFilters, CombineUnions, ConstantFolding, EliminateLimits, EliminateOuterJoin, EliminateSerialization, EliminateSorts, FoldablePropagation, NullPropagation, PushDownPredicates, PushPredicateThroughJoin, PushProjectionThroughUnion, RemoveDispensableExpressions, RemoveRedundantAliases, ReorderAssociativeOperator, ReorderJoin, RewriteCorrelatedScalarSubquery, SimplifyBinaryComparison, SimplifyCaseConversionExpressions, SimplifyCasts, SimplifyConditionals}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Join, LogicalPlan, Statistics, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+import org.apache.carbondata.mv.plans.util.BirdcageOptimizer
+
+object SparkVersionHelper {
+  def getStatisticsObj(outputList: Seq[NamedExpression],
+      plan: LogicalPlan, stats: Statistics,
+      aliasMap: Option[AttributeMap[Attribute]] = None): Statistics = {
+    val output = outputList.map(_.toAttribute)
+    val mapSeq = plan.collect { case n: logical.LeafNode => n }.map {
+      table => AttributeMap(table.output.zip(output))
+    }
+    val rewrites = mapSeq.head
+    val attributes: AttributeMap[ColumnStat] = stats.attributeStats
+    var attributeStats = AttributeMap(attributes.iterator
+      .map { pair => (rewrites(pair._1), pair._2) }.toSeq)
+    if (aliasMap.isDefined) {
+      attributeStats = AttributeMap(
+        attributeStats.map(pair => (aliasMap.get(pair._1), pair._2)).toSeq)
+    }
+    Statistics(stats.sizeInBytes, stats.rowCount, attributeStats)
+  }
+
+  def getOptimizedPlan(s: SubqueryExpression): LogicalPlan = {
+    val Subquery(newPlan, _) = BirdcageOptimizer.execute(Subquery.fromExpression(s))
+    newPlan
+  }
+
+  def normalizeExpressions[T <: Expression](r: T, attrs: AttributeSeq): T = {
+    QueryPlan.normalizeExpressions(r, attrs)
+  }
+
+  def attributeMap(rAliasMap: AttributeMap[Attribute]) : AttributeMap[Alias] = {
+    rAliasMap.asInstanceOf[AttributeMap[Alias]]
+  }
+
+  def seqOfRules : Seq[Rule[LogicalPlan]] = {
+    Seq(
+      // Operator push down
+      PushProjectionThroughUnion,
+      ReorderJoin,
+      EliminateOuterJoin,
+      PushPredicateThroughJoin,
+      PushDownPredicates,
+      ColumnPruning,
+      // Operator combine
+      CollapseRepartition,
+      CollapseProject,
+      CollapseWindow,
+      CombineFilters,
+      EliminateLimits,
+      CombineUnions,
+      // Constant folding and strength reduction
+      NullPropagation,
+      FoldablePropagation,
+      ConstantFolding,
+      ReorderAssociativeOperator,
+      // No need to apply LikeSimplification rule while creating MV
+      // as modular plan asCompactSql will be set in schema
+      //        LikeSimplification,
+      BooleanSimplification,
+      SimplifyConditionals,
+      RemoveDispensableExpressions,
+      SimplifyBinaryComparison,
+      EliminateSorts,
+      SimplifyCasts,
+      SimplifyCaseConversionExpressions,
+      RewriteCorrelatedScalarSubquery,
+      EliminateSerialization,
+      RemoveRedundantAliases)
+  }
+
+}
+
+trait getVerboseString extends LeafNode {
+  def verboseString: String = toString
+}
+
+trait groupByUnaryNode extends UnaryNode {
+  override def verboseString(maxFields: Int): String = super.verboseString(maxFields)
+
+  override def mapProductIterator[B](f: Any => B)(implicit evidence$1: ClassTag[B]): Array[B] = {
+    super.mapProductIterator(f)
+  }
+
+  override def mapChildren(f: ModularPlan => ModularPlan) : ModularPlan = {
+    val groupBy = super.mapChildren(f)
+    if (this.rewritten && !groupBy.rewritten) {
+      groupBy.setRewritten()
+    }
+    groupBy
+  }
+}
+
+trait selectModularPlan extends ModularPlan {
+  override def verboseString(maxFields: Int): String = super.verboseString(maxFields)
+
+  override def mapProductIterator[B](f: Any => B)(implicit evidence$1: ClassTag[B]): Array[B] = {
+    super.mapProductIterator(f)
+  }
+
+  override def mapChildren(f: ModularPlan => ModularPlan) : ModularPlan = {
+    val select = super.mapChildren(f)
+    if (this.rewritten && !select.rewritten) {
+      select.setRewritten()
+    }
+    select
+  }
+}
+
+trait unionModularPlan extends ModularPlan {

Review comment:
       please change all trait names, according to above @kunal642 comment

##########
File path: integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/CarbonDataSourceScanHelper.scala
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.spark.sql
+
+import org.apache.spark.CarbonInputMetrics
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression => SparkExpression}
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.execution.{ColumnarBatchScan, DataSourceScanExec}
+import org.apache.spark.sql.execution.strategy.CarbonPlanHelper
+import org.apache.spark.sql.optimizer.CarbonFilters
+
+import org.apache.carbondata.core.index.IndexFilter
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.scan.expression.Expression
+import org.apache.carbondata.core.scan.expression.logical.AndExpression
+import org.apache.carbondata.hadoop.CarbonProjection
+import org.apache.carbondata.spark.rdd.CarbonScanRDD
+
+abstract class  CarbonDataSourceScanHelper(relation: CarbonDatasourceHadoopRelation,
+                                           output: Seq[Attribute],
+    partitionFilters: Seq[SparkExpression],
+    pushedDownFilters: Seq[Expression],
+    pushedDownProjection: CarbonProjection,
+    directScanSupport: Boolean,
+    extraRDD: Option[(RDD[InternalRow], Boolean)],
+    segmentIds: Option[String])
+  extends DataSourceScanExec with ColumnarBatchScan {

Review comment:
       please correct the style of class definition, please also check all the places, if I miss somewhere

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, Expression, ExprId, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, ShuffledRowRDD, SimpleMode, SparkPlan, SQLExecution, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, RefreshTableCommand, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FilePartition, PartitionedFile}
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.hive.HiveExternalCatalog
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.types.{AbstractDataType, CharType, DataType, Metadata, StringType, StructField, VarcharType}
+import org.apache.spark.sql.util.SparkSQLUtil

Review comment:
       there are so many unused imports, please remove and check in other places and also check why the CI didn't fail for this

##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala
##########
@@ -21,21 +21,20 @@ import java.util.concurrent.Callable
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier}
 import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
 import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition, FunctionResourceLoader, GlobalTempViewManager}
 import org.apache.spark.sql.catalyst.expressions.Expression
 import org.apache.spark.sql.catalyst.parser.ParserInterface
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier}
 import org.apache.spark.sql.execution.strategy.{CarbonSourceStrategy, DDLStrategy, DMLStrategy, StreamingTableStrategy}
 import org.apache.spark.sql.hive.client.HiveClient
 import org.apache.spark.sql.internal.{SessionState, SQLConf}
 import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule}
 import org.apache.spark.sql.parser.CarbonSparkSqlParser
-import org.apache.spark.sql.{CarbonEnv, SparkSession}
 
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 

Review comment:
       revert changes if not required

##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonBoundReference.scala
##########
@@ -18,8 +18,8 @@
 package org.apache.spark.sql
 
 import org.apache.spark.sql.catalyst.InternalRow
-import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
 import org.apache.spark.sql.catalyst.expressions.{Attribute, ExprId, LeafExpression, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
 import org.apache.spark.sql.types.DataType
 

Review comment:
       revert this change

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
##########
@@ -544,12 +545,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     }
 
   protected lazy val explainPlan: Parser[LogicalPlan] =
-    (EXPLAIN ~> opt(EXTENDED)) ~ start ^^ {
-      case isExtended ~ logicalPlan =>
+    (EXPLAIN ~> opt(MODE)) ~ start ^^ {
+      case mode ~ logicalPlan =>

Review comment:
       can you please add test cases with different modes?

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkVersionAdapter.scala
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener, SessionCatalog}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, Expression, ExpressionSet, ExprId, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode}
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, ShuffledRowRDD, SimpleMode, SparkPlan, SQLExecution, UnaryExecNode}

Review comment:
       many unused imports, please remove

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/addsegment/AddSegmentTestCase.scala
##########
@@ -298,7 +301,7 @@ class AddSegmentTestCase extends QueryTest with BeforeAndAfterAll {
     for (i <- 0 until 10) {
       sql(s"alter table addsegment1 add segment " +
           s"options('path'='${ newPath + i }', 'format'='carbon')").collect()
-
+      sql("select count(*) from addsegment1").show()

Review comment:
       revert this

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/integration/spark/testsuite/binary/TestBinaryDataType.scala
##########
@@ -1551,26 +1552,28 @@ class TestBinaryDataType extends QueryTest with BeforeAndAfterAll {
             sql("insert into hive_table select 'a','b','1'");
         }
 
-        val e2 = intercept[SparkException] {
-            sql("insert into hive_table2 select 'a','b','binary'");
-        }
+        if (!SparkUtil.isSparkVersionXAndAbove("3")) {
+            val e2 = intercept[SparkException] {
+                sql("insert into hive_table2 select 'a','b','binary'");
+            }
 
-        assert(e2.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e2.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))
 
-        val eInt2 = intercept[Exception] {
-            sql("insert into hive_table2 select 'a','b','1'");
-        }
+            val eInt2 = intercept[Exception] {
+                sql("insert into hive_table2 select 'a','b','1'");
+            }
 
-        val e3 = intercept[SparkException] {
-            sql("insert into parquet_table select 'a','b','binary'");
-        }
+            val e3 = intercept[SparkException] {
+                sql("insert into parquet_table select 'a','b','binary'");
+            }
 
-        assert(e3.getMessage.contains(
-            "Dynamic partition strict mode requires at least one static partition column"))
+            assert(e3.getMessage.contains(
+                "Dynamic partition strict mode requires at least one static partition column"))

Review comment:
       revert all the above space changes

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFileCommand.scala
##########
@@ -100,6 +100,7 @@ class TestCleanFileCommand extends QueryTest with BeforeAndAfterAll {
   test("clean up table and test trash folder with Marked For Delete and Compacted segments") {
     // do not send MFD folders to trash
     createTable()
+    sql(s"""Show Tables """).show()

Review comment:
       revert this

##########
File path: mv/plan/src/main/common2.3and2.4/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.mv.plans.modular
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, Expression, ExprId, NamedExpression, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, CollapseProject, CollapseRepartition, CollapseWindow, ColumnPruning, CombineFilters, CombineLimits, CombineUnions, ConstantFolding, EliminateOuterJoin, EliminateSerialization, EliminateSorts, FoldablePropagation, NullPropagation, PushDownPredicate, PushPredicateThroughJoin, PushProjectionThroughUnion, RemoveDispensableExpressions, RemoveRedundantAliases, RemoveRedundantProject, ReorderAssociativeOperator, ReorderJoin, RewriteCorrelatedScalarSubquery, SimplifyBinaryComparison, SimplifyCaseConversionExpressions, SimplifyCasts, SimplifyConditionals}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Join, LogicalPlan, Statistics, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{DataType, Metadata}
+
+import org.apache.carbondata.mv.plans.util.BirdcageOptimizer
+
+
+object SparkVersionHelper {
+
+  def getStatisticsObj(outputList: Seq[NamedExpression],
+                       plan: LogicalPlan, stats: Statistics,
+                       aliasMap: Option[AttributeMap[Attribute]] = None): Statistics = {
+    val output = outputList.map(_.toAttribute)

Review comment:
       please correct the style and do for all code of class

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/merge/MergeTestCase.scala
##########
@@ -842,7 +842,7 @@ class MergeTestCase extends QueryTest with BeforeAndAfterAll {
       Row("d", "3")
     ).asJava, StructType(Seq(StructField("key", StringType), StructField("value", StringType))))
 
-    initframe.write
+    initframe.repartition(1).write

Review comment:
       why this change

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkVersionAdapter.scala
##########
@@ -0,0 +1,549 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener, SessionCatalog}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, Expression, ExpressionSet, ExprId, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode}
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, ShuffledRowRDD, SimpleMode, SparkPlan, SQLExecution, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, RefreshTableCommand, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.DataSourceStrategy
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.metric.SQLShuffleWriteMetricsReporter
+import org.apache.spark.sql.execution.strategy.CarbonDataSourceScan
+import org.apache.spark.sql.internal.{SessionState, SharedState}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.parser.CarbonSparkSqlParserUtil.{checkIfDuplicateColumnExists, convertDbNameToLowerCase, validateStreamingProperty}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{AbstractDataType, CharType, DataType, Metadata, StringType, StructField, VarcharType}
+import org.apache.spark.sql.util.SparkSQLUtil
+import org.apache.spark.unsafe.types.UTF8String
+
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.SchemaReader
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+import org.apache.carbondata.spark.CarbonOption
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+object SparkVersionAdapter {
+
+  def getExplainCommandObj(logicalPlan: LogicalPlan = OneRowRelation(),
+                           mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(logicalPlan, ExplainMode.fromString(mode.getOrElse(SimpleMode.name)))
+  }
+
+  def getExplainCommandObj(mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(OneRowRelation(), ExplainMode.fromString(mode.getOrElse(SimpleMode.name)))
+  }
+
+  /**
+   * As a part of SPARK-24085 Hive tables supports scala subquery for
+   * the partitioned tables,so Carbon also needs to supports
+   * @param partitionSet
+   * @param filterPredicates
+   * @return
+   */
+  def getPartitionFilter(
+                          partitionSet: AttributeSet,
+                          filterPredicates: Seq[Expression]): Seq[Expression] = {
+    filterPredicates
+      .filterNot(SubqueryExpression.hasSubquery)
+      .filter { filter =>
+        filter.references.nonEmpty && filter.references.subsetOf(partitionSet)
+      }
+  }
+
+  def getDataFilter(partitionSet: AttributeSet, filter: Seq[Expression]): Seq[Expression] = {
+    filter
+  }
+
+  // As per SPARK-22520 OptimizeCodegen is removed in 2.3.1
+  def getOptimizeCodegenRule(): Seq[Rule[LogicalPlan]] = {
+    Seq.empty
+  }
+
+  def getUpdatedStorageFormat(storageFormat: CatalogStorageFormat,
+                              map: Map[String, String],
+                              tablePath: String): CatalogStorageFormat = {
+    storageFormat.copy(properties = map, locationUri = Some(new URI(tablePath)))
+  }
+
+  def getOutput(subQueryAlias: SubqueryAlias): Seq[Attribute] = {
+    val newAlias = Seq(subQueryAlias.identifier.name)
+    subQueryAlias.child.output.map(_.withQualifier(newAlias))
+  }
+
+  def stringToTimestamp(timestamp: String): Option[Long] = {
+    DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp), ZoneId.systemDefault())
+  }
+
+  def stringToTime(value: String): java.util.Date = {
+    stringToDateValue(value)
+  }
+
+  def getPredicate(inputSchema: Seq[Attribute],
+                   condition: Option[Expression]): InternalRow => Boolean = {
+    Predicate.create(condition.get, inputSchema).eval _
+  }
+
+  @tailrec
+  private def stringToDateValue(value: String): java.util.Date = {
+    val indexOfGMT = value.indexOf("GMT")
+    if (indexOfGMT != -1) {
+      // ISO8601 with a weird time zone specifier (2000-01-01T00:00GMT+01:00)
+      val s0 = value.substring(0, indexOfGMT)
+      val s1 = value.substring(indexOfGMT + 3)
+      // Mapped to 2000-01-01T00:00+01:00
+      stringToDateValue(s0 + s1)
+    } else if (!value.contains('T')) {
+      // JDBC escape string
+      if (value.contains(' ')) {
+        Timestamp.valueOf(value)
+      } else {
+        Date.valueOf(value)
+      }
+    } else {
+      DatatypeConverter.parseDateTime(value).getTime
+    }
+  }
+
+  def timeStampToString(timeStamp: Long): String = {
+    TimestampFormatter.getFractionFormatter(ZoneId.systemDefault()).format(timeStamp)
+  }
+
+  def dateToString(date: Int): String = {
+    DateTimeUtils.daysToLocalDate(date).toString
+  }
+
+  def addTaskCompletionListener[U](f: => U) {
+    TaskContext.get().addTaskCompletionListener[Unit] { context =>
+      f
+    }
+  }
+
+  def getTableIdentifier(u: UnresolvedRelation): Some[TableIdentifier] = {
+    val tableName = u.tableName.split("\\.")
+    if (tableName.size == 2) {
+      Some(TableIdentifier(tableName(1), Option(tableName(0))))
+    } else {
+      val currentDatabase = SparkSQLUtil.getSparkSession.sessionState.catalog.getCurrentDatabase
+      Some(TableIdentifier(tableName(0), Option(currentDatabase)))
+    }
+  }
+
+  def createRangeListScalaUDF(toRangeListUDF: ToRangeListAsStringUDF,
+                              dataType: StringType.type,
+                              children: Seq[Expression],
+                              inputTypes: Seq[DataType]): ScalaUDF = {
+    ScalaUDF(toRangeListUDF,
+      dataType,
+      children,
+      Nil,
+      None,
+      Some("ToRangeListAsString"))
+  }
+
+  def getTransformedPolygonJoinUdf(scalaUdf: ScalaUDF,
+                                   udfChildren: Seq[Expression],
+                                   polygonJoinUdf: InPolygonJoinUDF): ScalaUDF = {
+    ScalaUDF(polygonJoinUdf,
+      scalaUdf.dataType,
+      udfChildren,
+      scalaUdf.inputEncoders,
+      scalaUdf.outputEncoder,
+      scalaUdf.udfName)
+  }
+
+  def getTableIdentifier(parts: Seq[String]): TableIdentifier = {
+    if (parts.length == 1) {
+      TableIdentifier(parts.head, None)
+    } else {
+      TableIdentifier(parts(1), Option(parts.head))
+    }
+  }
+
+  def createShuffledRowRDD(sparkContext: SparkContext, localTopK: RDD[InternalRow],
+                           child: SparkPlan, serializer: Serializer): ShuffledRowRDD = {
+    val writeMetrics = SQLShuffleWriteMetricsReporter.createShuffleWriteMetrics(sparkContext)
+    new ShuffledRowRDD(
+      ShuffleExchangeExec.prepareShuffleDependency(
+        localTopK, child.output, SinglePartition, serializer, writeMetrics), writeMetrics)
+  }
+
+  def invokeAnalyzerExecute(analyzer: Analyzer,
+                            plan: LogicalPlan): LogicalPlan = {
+    analyzer.executeAndCheck(plan, QueryPlanningTracker.get.getOrElse(new QueryPlanningTracker))
+  }
+
+  def normalizeExpressions[T <: Expression](r: T, attrs: AttributeSeq): T = {
+    QueryPlan.normalizeExpressions(r, attrs)
+  }
+
+  def getBuildRight: BuildSide = {
+    BuildRight
+  }
+
+  def getBuildLeft: BuildSide = {
+    BuildLeft
+  }
+
+  def withNewExecutionId[T](sparkSession: SparkSession, queryExecution: QueryExecution): T => T = {
+    SQLExecution.withNewExecutionId(queryExecution, None)(_)
+  }
+
+  def createJoinNode(child: LogicalPlan,
+                     targetTable: LogicalPlan,
+                     joinType: JoinType,
+                     condition: Option[Expression]): Join = {
+    Join(child, targetTable, joinType, condition, JoinHint.NONE)
+  }
+
+  def getPartitionsFromInsert(x: InsertIntoStatementWrapper): Map[String, Option[String]] = {
+    x.partitionSpec
+  }
+
+  type CarbonBuildSideType = BuildSide
+  type InsertIntoStatementWrapper = InsertIntoStatement
+
+  def createRefreshTableCommand(tableIdentifier: TableIdentifier): RefreshTableCommand = {
+    RefreshTableCommand(tableIdentifier)
+  }
+
+  type RefreshTables = RefreshTableCommand
+
+  /**
+   * Validates the partition columns and return's A tuple of partition columns and partitioner
+   * fields.
+   *
+   * @param partitionColumns        An instance of ColTypeListContext having parser rules for
+   *                                column.
+   * @param colNames                <Seq[String]> Sequence of Table column names.
+   * @param tableProperties         <Map[String, String]> Table property map.
+   * @param partitionByStructFields Seq[StructField] Sequence of partition fields.
+   * @return <Seq[PartitionerField]> A Seq of partitioner fields.
+   */
+  def validatePartitionFields(
+                               partitionColumns: PartitionFieldListContext,
+                               colNames: Seq[String],
+                               tableProperties: mutable.Map[String, String],
+                               partitionByStructFields: Seq[StructField]): Seq[PartitionerField] = {
+
+    val partitionerFields = partitionByStructFields.map { structField =>
+      PartitionerField(structField.name, Some(structField.dataType.toString), null)
+    }
+    // validate partition clause
+    if (partitionerFields.nonEmpty) {
+      // partition columns should not be part of the schema
+      val badPartCols = partitionerFields.map(_.partitionColumn.toLowerCase).toSet
+        .intersect(colNames.map(_.toLowerCase).toSet)
+      if (badPartCols.nonEmpty) {
+        operationNotAllowed(s"Partition columns should not be specified in the schema: " +
+          badPartCols.map("\"" + _ + "\"").mkString("[", ",", "]")
+          , partitionColumns: PartitionFieldListContext)
+      }
+    }
+    partitionerFields
+  }
+
+  /**
+   * The method validates the create table command and returns the create table or
+   * ctas table LogicalPlan.
+   *
+   * @param createTableTuple a tuple of (CreateTableHeaderContext, SkewSpecContext,
+   *                         BucketSpecContext, PartitionFieldListContext, ColTypeListContext,
+   *                         TablePropertyListContext,
+   *                         LocationSpecContext, Option[String], TerminalNode, QueryContext,
+   *                         String)
+   * @param extraTableTuple  A tuple of (Seq[StructField], Boolean, TableIdentifier, Boolean,
+   *                         Seq[String],
+   *                         Option[String], mutable.Map[String, String], Map[String, String],
+   *                         Seq[StructField],
+   *                         Seq[PartitionerField], CarbonSpark2SqlParser, SparkSession,
+   *                         Option[LogicalPlan])
+   * @return <LogicalPlan> of create table or ctas table
+   *
+   */
+  def createCarbonTable(createTableTuple: (CreateTableHeaderContext, SkewSpecContext,
+    BucketSpecContext, PartitionFieldListContext, ColTypeListContext, TablePropertyListContext,
+    LocationSpecContext, Option[String], TerminalNode, QueryContext, String),
+    extraTableTuple: (Seq[StructField], Boolean, TableIdentifier, Boolean, Seq[String],
+    Option[String], mutable.Map[String, String], Map[String, String], Seq[StructField],
+    Seq[PartitionerField], CarbonSpark2SqlParser, SparkSession,
+    Option[LogicalPlan])): LogicalPlan = {
+    val (tableHeader, skewSpecContext, bucketSpecContext, partitionColumns, columns,
+    tablePropertyList, locationSpecContext, tableComment, ctas, query, provider) = createTableTuple
+    val (cols, external, tableIdentifier, ifNotExists, colNames, tablePath,
+    tableProperties, properties, partitionByStructFields, partitionFields,
+    parser, sparkSession, selectQuery) = extraTableTuple
+    val options = new CarbonOption(properties)
+    // validate streaming property
+    validateStreamingProperty(options)
+    var fields = parser.getFields(cols ++ partitionByStructFields)
+    // validate for create table as select
+    selectQuery match {
+      case Some(q) =>
+        // create table as select does not allow creation of partitioned table
+        if (partitionFields.nonEmpty) {
+          val errorMessage = "A Create Table As Select (CTAS) statement is not allowed to " +
+            "create a partitioned table using Carbondata file formats."
+          operationNotAllowed(errorMessage, partitionColumns)
+        }
+        // create table as select does not allow to explicitly specify schema
+        if (fields.nonEmpty) {
+          operationNotAllowed(
+            "Schema may not be specified in a Create Table As Select (CTAS) statement", columns)
+        }
+        // external table is not allow
+        if (external) {
+          operationNotAllowed("Create external table as select", tableHeader)
+        }
+        fields = parser
+          .getFields(CarbonEnv.getInstance(sparkSession).carbonMetaStore
+            .getSchemaFromUnresolvedRelation(sparkSession, Some(q).get))
+      case _ =>
+      // ignore this case
+    }
+    val columnNames = fields.map(_.name.get)
+    checkIfDuplicateColumnExists(columns, tableIdentifier, columnNames)
+    if (partitionFields.nonEmpty && options.isStreaming) {
+      operationNotAllowed("Streaming is not allowed on partitioned table", partitionColumns)
+    }
+
+    if (!external && fields.isEmpty) {
+      throw new MalformedCarbonCommandException("Creating table without column(s) is not supported")
+    }
+    if (external && fields.isEmpty && tableProperties.nonEmpty) {
+      // as fields are always zero for external table, cannot validate table properties.
+      operationNotAllowed(
+        "Table properties are not supported for external table", tablePropertyList)
+    }
+
+    // Global dictionary is deprecated since 2.0
+    if (tableProperties.contains(CarbonCommonConstants.DICTIONARY_INCLUDE) ||
+      tableProperties.contains(CarbonCommonConstants.DICTIONARY_EXCLUDE)) {
+      DeprecatedFeatureException.globalDictNotSupported()
+    }
+
+    val bucketFields = parser.getBucketFields(tableProperties, fields, options)
+    var isTransactionalTable: Boolean = true
+
+    val tableInfo = if (external) {
+      if (fields.nonEmpty) {
+        // user provided schema for this external table, this is not allow currently
+        // see CARBONDATA-2866
+        operationNotAllowed(
+          "Schema must not be specified for external table", columns)
+      }
+      if (partitionByStructFields.nonEmpty) {
+        operationNotAllowed(
+          "Partition is not supported for external table", partitionColumns)
+      }
+      // read table info from schema file in the provided table path
+      // external table also must convert table name to lower case
+      val identifier = AbsoluteTableIdentifier.from(
+        tablePath.get,
+        CarbonEnv.getDatabaseName(tableIdentifier.database)(sparkSession).toLowerCase(),
+        tableIdentifier.table.toLowerCase())
+      val table = try {
+        val schemaPath = CarbonTablePath.getSchemaFilePath(identifier.getTablePath)
+        if (!FileFactory.isFileExist(schemaPath)) {
+          if (provider.equalsIgnoreCase("'carbonfile'")) {
+            SchemaReader.inferSchema(identifier, true)
+          } else {
+            isTransactionalTable = false
+            SchemaReader.inferSchema(identifier, false)
+          }
+        } else {
+          SchemaReader.getTableInfo(identifier)
+        }
+      } catch {
+        case e: Throwable =>
+          operationNotAllowed(s"Invalid table path provided: ${ tablePath.get } ", tableHeader)
+      }
+
+      // set "_external" property, so that DROP TABLE will not delete the data
+      if (provider.equalsIgnoreCase("'carbonfile'")) {
+        table.getFactTable.getTableProperties.put("_filelevelformat", "true")
+        table.getFactTable.getTableProperties.put("_external", "false")
+      } else {
+        table.getFactTable.getTableProperties.put("_external", "true")
+        table.getFactTable.getTableProperties.put("_filelevelformat", "false")
+      }
+      var isLocalDic_enabled = table.getFactTable.getTableProperties
+        .get(CarbonCommonConstants.LOCAL_DICTIONARY_ENABLE)
+      if (null == isLocalDic_enabled) {
+        table.getFactTable.getTableProperties
+          .put(CarbonCommonConstants.LOCAL_DICTIONARY_ENABLE,
+            CarbonProperties.getInstance()
+              .getProperty(CarbonCommonConstants.LOCAL_DICTIONARY_SYSTEM_ENABLE,
+                CarbonCommonConstants.LOCAL_DICTIONARY_ENABLE_DEFAULT))
+      }
+      isLocalDic_enabled = table.getFactTable.getTableProperties
+        .get(CarbonCommonConstants.LOCAL_DICTIONARY_ENABLE)
+      if (CarbonScalaUtil.validateLocalDictionaryEnable(isLocalDic_enabled) &&
+        isLocalDic_enabled.toBoolean) {
+        val allColumns = table.getFactTable.getListOfColumns
+        for (i <- 0 until allColumns.size()) {
+          val cols = allColumns.get(i)
+          if (cols.getDataType == DataTypes.STRING || cols.getDataType == DataTypes.VARCHAR) {
+            cols.setLocalDictColumn(true)
+          }
+        }
+        table.getFactTable.setListOfColumns(allColumns)
+      }
+      table
+    } else {
+      // prepare table model of the collected tokens
+      val tableModel: TableModel = CarbonParserUtil.prepareTableModel(
+        ifNotExists,
+        convertDbNameToLowerCase(tableIdentifier.database),
+        tableIdentifier.table.toLowerCase,
+        fields,
+        partitionFields,
+        tableProperties,
+        bucketFields,
+        isAlterFlow = false,
+        tableComment)
+      TableNewProcessor(tableModel)
+    }
+    tableInfo.setTransactionalTable(isTransactionalTable)
+    selectQuery match {
+      case query@Some(q) =>
+        CarbonCreateTableAsSelectCommand(
+          tableInfo = tableInfo,
+          query = query.get,
+          ifNotExistsSet = ifNotExists,
+          tableLocation = tablePath)
+      case _ =>
+        CarbonCreateTableCommand(
+          tableInfo = tableInfo,
+          ifNotExistsSet = ifNotExists,
+          tableLocation = tablePath,
+          external)
+    }
+  }
+
+  def getField(parser: CarbonSpark2SqlParser,
+               schema: Seq[QualifiedColType], isExternal: Boolean = false): Seq[Field] = {
+    schema.map { col =>
+      parser.getFields(col.comment, col.name.head, col.dataType, isExternal)
+    }
+  }
+
+  def supportsBatchOrColumnar(scan: CarbonDataSourceScan): Boolean = {
+    scan.supportsColumnar
+  }
+
+  def createDataset(sparkSession: SparkSession, qe: QueryExecution) : Dataset[Row] = {
+    new Dataset[Row](qe, RowEncoder(qe.analyzed.schema))
+  }
+
+  def createSharedState(sparkContext: SparkContext) : SharedState = {
+    new SharedState(sparkContext, Map.empty[String, String])
+  }
+
+  def translateFilter(dataFilters: Seq[Expression]) : Seq[Filter] = {
+    dataFilters.flatMap(DataSourceStrategy.translateFilter(_,
+      supportNestedPredicatePushdown = false))
+  }
+
+  def getCarbonOptimizer(session: SparkSession,
+                         sessionState: SessionState): CarbonOptimizer = {
+    new CarbonOptimizer(session, sessionState.optimizer)
+  }
+
+  def isCharType(dataType: DataType): Boolean = {
+    dataType.isInstanceOf[CharType]
+  }
+
+  def isVarCharType(dataType: DataType): Boolean = {
+    dataType.isInstanceOf[VarcharType]
+  }
+
+  def getTypeName(s: AbstractDataType): String = {
+    s.defaultConcreteType.typeName
+  }
+
+  def getInsertIntoCommand(table: LogicalPlan,
+                           partition: Map[String, Option[String]],
+                           query: LogicalPlan,
+                           overwrite: Boolean,
+                           ifPartitionNotExists: Boolean): InsertIntoStatement = {
+    InsertIntoStatement(
+      table,
+      partition,
+      Nil,
+      query,
+      overwrite,
+      ifPartitionNotExists)
+  }
+}
+
+case class CarbonBuildSide(buildSide: BuildSide) {
+  def isRight: Boolean = buildSide.isInstanceOf[BuildRight.type]
+  def isLeft: Boolean = buildSide.isInstanceOf[BuildLeft.type]
+}
+
+abstract class CarbonTakeOrderedAndProjectExecHelper(sortOrder: Seq[SortOrder],
+    limit: Int, skipMapOrder: Boolean, readFromHead: Boolean) extends UnaryExecNode {
+  override def simpleString(maxFields: Int): String = {
+    val orderByString = sortOrder.mkString("[", ",", "]")
+    val outputString = output.mkString("[", ",", "]")
+
+    s"CarbonTakeOrderedAndProjectExec(limit=$limit, orderBy=$orderByString, " +
+      s"skipMapOrder=$skipMapOrder, readFromHead=$readFromHead, output=$outputString)"
+  }
+}

Review comment:
       add a new line after class, all these should be caught in CI, please check once

##########
File path: mv/plan/src/main/spark2.3/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala
##########
@@ -46,5 +46,4 @@ object ExpressionHelper {
   def getTheLastQualifier(reference: AttributeReference): String = {
     reference.qualifier.head
   }
-
 }

Review comment:
       Please revert this class changes, not needed

##########
File path: scalastyle-config.xml
##########
@@ -203,19 +203,6 @@ This file is divided into 3 sections:
     ]]></customMessage>
  </check>
 
- <check customId="awaitresult" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
-  <parameters>
-   <parameter name="regex">Await\.result</parameter>
-  </parameters>
-  <customMessage><![CDATA[
-      Are you sure that you want to use Await.result? In most cases, you should use ThreadUtils.awaitResult instead.
-      If you must use Await.result, wrap the code block with
-      // scalastyle:off awaitresult
-      Await.result(...)
-      // scalastyle:on awaitresult
-    ]]></customMessage>
- </check>
-

Review comment:
       why this removed?




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

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646308065



##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneIndexExample.scala
##########
@@ -60,40 +60,6 @@ object LuceneIndexExample {
          | AS 'lucene'
       """.stripMargin)
 
-    // 1. Compare the performance:
-
-    def time(code: => Unit): Double = {
-      val start = System.currentTimeMillis()
-      code
-      // return time in second
-      (System.currentTimeMillis() - start).toDouble / 1000
-    }
-
-    val timeWithoutLuceneIndex = time {
-
-      spark.sql(
-        s"""
-           | SELECT count(*)
-           | FROM personTable where id like '% test1 %'
-      """.stripMargin).show()
-
-    }
-
-    val timeWithLuceneIndex = time {
-
-      spark.sql(

Review comment:
       ignore this run in RunExamples.scala and revert the change 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:
users@infra.apache.org



[GitHub] [carbondata] kunal642 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-855624385


   LGTM
   
   @vikramahuja1001 please handle the comments and/or reply to the comments.. lets get this merged ASAP


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644563014



##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.spark.sql.execution.strategy
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.CarbonInputMetrics
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.CarbonDatasourceHadoopRelation
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonSparkDataSourceUtil
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, SortOrder, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.{Expression => SparkExpression}
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
+import org.apache.spark.sql.execution.{ColumnarBatchScan, DataSourceScanExec, WholeStageCodegenExec}
+import org.apache.spark.sql.optimizer.CarbonFilters
+import org.apache.spark.sql.types.AtomicType
+
+import org.apache.carbondata.core.index.IndexFilter
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.metadata.schema.BucketingInfo
+import org.apache.carbondata.core.readcommitter.ReadCommittedScope
+import org.apache.carbondata.core.scan.expression.Expression
+import org.apache.carbondata.core.scan.expression.logical.AndExpression
+import org.apache.carbondata.hadoop.CarbonProjection
+import org.apache.carbondata.spark.rdd.CarbonScanRDD
+
+/**
+ *  Physical plan node for scanning data. It is applied for both tables
+ *  USING carbondata and STORED AS carbondata.
+ */
+case class CarbonDataSourceScan(

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647101468



##########
File path: integration/flink/pom.xml
##########
@@ -30,6 +30,11 @@
             <artifactId>carbondata-format</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>com.thoughtworks.paranamer</groupId>

Review comment:
       done

##########
File path: integration/flink/pom.xml
##########
@@ -193,9 +220,7 @@
     <profiles>
         <profile>
             <id>spark-2.3</id>
-            <activation>
-                <activeByDefault>true</activeByDefault>
-            </activation>
+          <activation><activeByDefault>true</activeByDefault></activation>

Review comment:
       done

##########
File path: streaming/src/main/spark2.x/org.apache.carbondata.util/SparkStreamingUtil.scala
##########
@@ -15,14 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.carbondata.spark.adapter
+package org.apache.carbondata.util
 
-import scala.collection.mutable.ArrayBuffer
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
 
-import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile}
+object SparkStreamingUtil {

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-861408682


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


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

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



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646345275



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonExtensionSpark2SqlParser.scala
##########
@@ -56,4 +58,13 @@ class CarbonExtensionSpark2SqlParser extends CarbonSpark2SqlParser {
         CarbonSparkSqlParserUtil.loadDataNew(
           databaseNameOp, tableName, Option(optionsList), partitions, filePath, isOverwrite)
     }
+
+  /**
+   * REFRESH MATERIALIZED VIEW mv_name
+   */
+  private lazy val refreshTable: Parser[LogicalPlan] =

Review comment:
       Spark 3.1 has added Refresh table in SparkSqlParser and it fails with UnresolvedTableOrView. So, carbon need to handle it. @vikramahuja1001 check and remove from DDLStrategy if not used




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-852861111


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644561121



##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -185,8 +224,390 @@ object CarbonToSparkAdapter {
   def getHiveExternalCatalog(sparkSession: SparkSession): HiveExternalCatalog = {
     sparkSession.sessionState.catalog.externalCatalog.asInstanceOf[HiveExternalCatalog]
   }
+
+  def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]) = {
+    FilePartition(index, files.toArray.toSeq)
+  }
+
+  def stringToTimestamp(timestamp: String): Option[Long] = {
+    DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp))
+  }
+
+  def getTableIdentifier(u: UnresolvedRelation): Some[TableIdentifier] = {
+    Some(u.tableIdentifier)
+  }
+
+  def dateToString(date: Int): String = {
+    DateTimeUtils.dateToString(date.toString.toInt)
+  }
+
+  def timeStampToString(timeStamp: Long): String = {
+    DateTimeUtils.timestampToString(timeStamp)
+  }
+
+  def stringToTime(value: String): java.util.Date = {
+    DateTimeUtils.stringToTime(value)
+  }
+
+  def getProcessingTime: String => Trigger = {
+    Trigger.ProcessingTime
+  }
+
+  def addTaskCompletionListener[U](f: => U) {
+    TaskContext.get().addTaskCompletionListener { context =>
+      f
+    }
+  }
+
+  def createShuffledRowRDD(sparkContext: SparkContext, localTopK: RDD[InternalRow],
+      child: SparkPlan, serializer: Serializer): ShuffledRowRDD = {
+    new ShuffledRowRDD(
+      ShuffleExchangeExec.prepareShuffleDependency(
+        localTopK, child.output, SinglePartition, serializer))
+  }
+
+  def getInsertIntoCommand(table: LogicalPlan,
+      partition: Map[String, Option[String]],
+      query: LogicalPlan,
+      overwrite: Boolean,
+      ifPartitionNotExists: Boolean): InsertIntoTable = {
+    InsertIntoTable(
+      table,
+      partition,
+      query,
+      overwrite,
+      ifPartitionNotExists)
+  }
+
+  def getExplainCommandObj(logicalPlan: LogicalPlan = OneRowRelation(),
+      mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(logicalPlan, mode.isDefined)
+  }
+
+  def getExplainCommandObj(mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(OneRowRelation(), mode.isDefined)
+  }
+
+  def invokeAnalyzerExecute(analyzer: Analyzer,
+      plan: LogicalPlan): LogicalPlan = {
+    analyzer.executeAndCheck(plan)
+  }
+
+  def normalizeExpressions(r: NamedExpression, attrs: AttributeSeq): NamedExpression = {
+    QueryPlan.normalizeExprId(r, attrs)
+  }
+
+  def getBuildRight: BuildSide = {
+    BuildRight
+  }
+
+  def getBuildLeft: BuildSide = {
+    BuildLeft
+  }
+
+  type CarbonBuildSideType = BuildSide
+  type InsertIntoStatementWrapper = InsertIntoTable
+
+  def withNewExecutionId[T](sparkSession: SparkSession, queryExecution: QueryExecution): T => T = {
+    SQLExecution.withNewExecutionId(sparkSession, queryExecution)(_)
+  }
+
+  def getTableIdentifier(parts: TableIdentifier): TableIdentifier = {
+    parts
+  }
+
+  def createJoinNode(child: LogicalPlan,
+      targetTable: LogicalPlan,
+      joinType: JoinType,
+      condition: Option[Expression]): Join = {
+    Join(child, targetTable, joinType, condition)
+  }
+
+  def getPartitionsFromInsert(x: InsertIntoStatementWrapper): Map[String, Option[String]] = {
+    x.partition
+  }
+
+  def getStatisticsObj(outputList: Seq[NamedExpression],

Review comment:
       yes removed




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644561823



##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala
##########
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.hive.execution.command
 
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.{CarbonEnv, CarbonToSparkAdapter, Row, SparkSession}

Review comment:
       removed




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-865014326


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-859530478






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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655295026



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/CsvRDDHelper.scala
##########
@@ -26,7 +26,7 @@ import org.apache.hadoop.mapreduce.{TaskAttemptID, TaskType}
 import org.apache.hadoop.mapreduce.lib.input.{FileInputFormat, FileSplit}
 import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl
 import org.apache.spark.rdd.RDD
-import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.{CarbonToSparkAdapter, SparkSession}

Review comment:
       That old package has been renamed




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-864899792






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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655294747



##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneIndexExample.scala
##########
@@ -61,37 +61,37 @@ object LuceneIndexExample {
       """.stripMargin)
 
     // 1. Compare the performance:
-
-    def time(code: => Unit): Double = {
-      val start = System.currentTimeMillis()
-      code
-      // return time in second
-      (System.currentTimeMillis() - start).toDouble / 1000
-    }
-
-    val timeWithoutLuceneIndex = time {
-
-      spark.sql(
-        s"""
-           | SELECT count(*)
-           | FROM personTable where id like '% test1 %'
-      """.stripMargin).show()
-
-    }
-
-    val timeWithLuceneIndex = time {
-
-      spark.sql(
-        s"""
-           | SELECT count(*)
-           | FROM personTable where TEXT_MATCH('id:test1')
-      """.stripMargin).show()
-
-    }
+    // TODO: Revert this after SPARK 3.1 fix

Review comment:
       changed this




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-851478977


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-850335259


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


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

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655123704



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/TestIndexCommand.scala
##########
@@ -39,7 +39,9 @@ class TestIndexCommand extends QueryTest with BeforeAndAfterAll {
 
   val newClass = "org.apache.spark.sql.CarbonSource"
 
-  test("test index create: don't support using non-exist class") {
+  // TODO: This issue will be fixed during alter change column support for spark 3.1.1
+  // REFER: https://issues.apache.org/jira/browse/CARBONDATA-4210
+  ignore("test index create: don't support using non-exist class") {

Review comment:
       why are these tests commented?

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/TestIndexCommand.scala
##########
@@ -39,7 +39,9 @@ class TestIndexCommand extends QueryTest with BeforeAndAfterAll {
 
   val newClass = "org.apache.spark.sql.CarbonSource"
 
-  test("test index create: don't support using non-exist class") {
+  // TODO: This issue will be fixed during alter change column support for spark 3.1.1
+  // REFER: https://issues.apache.org/jira/browse/CARBONDATA-4210
+  ignore("test index create: don't support using non-exist class") {

Review comment:
       why are these tests ignored?

##########
File path: integration/spark/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
##########
@@ -38,13 +38,13 @@ import org.apache.carbondata.spark.exception.ProcessMetaDataException
 
 class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll {
 
-  var threshold: Int = _
+  var threshold: String = _
 
   override def beforeAll {
 
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
-    threshold = sqlContext.getConf("spark.sql.autoBroadcastJoinThreshold").toInt
+    threshold = sqlContext.getConf("spark.sql.autoBroadcastJoinThreshold")

Review comment:
       use SQLConf.AUTO_BROADCASTJOIN_THRESHOLD




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647122952



##########
File path: scalastyle-config.xml
##########
@@ -203,19 +203,6 @@ This file is divided into 3 sections:
     ]]></customMessage>
  </check>
 
- <check customId="awaitresult" level="error" class="org.scalastyle.file.RegexChecker" enabled="true">
-  <parameters>
-   <parameter name="regex">Await\.result</parameter>
-  </parameters>
-  <customMessage><![CDATA[
-      Are you sure that you want to use Await.result? In most cases, you should use ThreadUtils.awaitResult instead.
-      If you must use Await.result, wrap the code block with
-      // scalastyle:off awaitresult
-      Await.result(...)
-      // scalastyle:on awaitresult
-    ]]></customMessage>
- </check>
-

Review comment:
       this was giving error at random places, thus removed it.




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-857662096


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644556914



##########
File path: mv/plan/src/main/spark2.3/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala
##########
@@ -47,4 +55,110 @@ object ExpressionHelper {
     reference.qualifier.head
   }
 
+  def getStatisticsObj(outputList: Seq[NamedExpression],
+      plan: LogicalPlan, stats: Statistics,
+      aliasMap: Option[AttributeMap[Attribute]] = None): Statistics = {
+    val output = outputList.map(_.toAttribute)
+    val mapSeq = plan.collect { case n: logical.LeafNode => n }.map {
+      table => AttributeMap(table.output.zip(output))
+    }
+    val rewrites = mapSeq.head
+    val attributes: AttributeMap[ColumnStat] = stats.attributeStats
+    var attributeStats = AttributeMap(attributes.iterator
+      .map { pair => (rewrites(pair._1), pair._2) }.toSeq)
+    if (aliasMap.isDefined) {
+      attributeStats = AttributeMap(
+        attributeStats.map(pair => (aliasMap.get(pair._1), pair._2)).toSeq)
+    }
+    Statistics(stats.sizeInBytes, stats.rowCount, attributeStats, stats.hints)
+  }
+
+  def getOptimizedPlan(s: SubqueryExpression): LogicalPlan = {
+    val Subquery(newPlan) = BirdcageOptimizer.execute(Subquery(s.plan))
+    newPlan
+  }
+
+  def normalizeExpressions(r: NamedExpression, attrs: AttributeSeq): NamedExpression = {
+    QueryPlan.normalizeExprId(r, attrs)
+  }
+
+  def attributeMap(rAliasMap: AttributeMap[Attribute]) : AttributeMap[Expression] = {
+    rAliasMap.asInstanceOf[AttributeMap[Expression]]
+  }
+
+  def seqOfRules : Seq[Rule[LogicalPlan]] = {
+    Seq(
+      // Operator push down
+      PushProjectionThroughUnion,
+      ReorderJoin,
+      EliminateOuterJoin,
+      PushPredicateThroughJoin,
+      PushDownPredicate,
+      ColumnPruning,
+      // Operator combine
+      CollapseRepartition,
+      CollapseProject,
+      CollapseWindow,
+      CombineFilters,
+      CombineLimits,
+      CombineUnions,
+      // Constant folding and strength reduction
+      NullPropagation,
+      FoldablePropagation,
+      ConstantFolding,
+      ReorderAssociativeOperator,
+      // No need to apply LikeSimplification rule while creating MV
+      // as modular plan asCompactSql will be set in schema
+      //        LikeSimplification,
+      BooleanSimplification,
+      SimplifyConditionals,
+      RemoveDispensableExpressions,
+      SimplifyBinaryComparison,
+      EliminateSorts,
+      SimplifyCasts,
+      SimplifyCaseConversionExpressions,
+      RewriteCorrelatedScalarSubquery,
+      EliminateSerialization,
+      RemoveRedundantAliases,
+      RemoveRedundantProject)
+  }
+}
+
+trait getVerboseString extends LeafNode {
+}
+
+trait groupByUnaryNode extends UnaryNode {
+}
+
+trait selectModularPlan extends ModularPlan {
+}
+
+trait unionModularPlan extends ModularPlan {
+}
+
+trait oneRowTableLeafNode extends LeafNode {
+}
+
+object MatchJoin {
+  def unapply(plan : LogicalPlan): Option[(LogicalPlan, LogicalPlan, JoinType, Option[Expression],
+    Option[Any])] = {
+    plan match {
+      case j@Join(left, right, joinType, condition) =>
+        val a = Some(left, right, joinType, condition, None)
+        a

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-848231109


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644521888



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -439,10 +445,31 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
     }
   }
 
+  private static long createTimeInstant(String dimensionValue, String dateFormat) {
+    String updatedDim = dimensionValue;
+    if (!dimensionValue.trim().contains(" ")) {
+      updatedDim += " 00:00:00";
+    }
+    Instant instant = Instant.from(ZonedDateTime
+        .of(LocalDateTime.parse(updatedDim, DateTimeFormatter.ofPattern(dateFormat)),
+        ZoneId.systemDefault()));
+    validateTimeStampRange(instant.getEpochSecond());
+    long us = Math.multiplyExact(instant.getEpochSecond(), 1000L);
+    return Math.addExact(us, instant.getNano() * 1000L);
+  }
+
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
     DateFormat dateFormatter = null;
     long timeValue;
+    if (Boolean.parseBoolean(CarbonProperties.getInstance().getProperty(CarbonCommonConstants
+        .CARBON_SPARK_VERSION_SPARK3))) {
+      try {
+        return createTimeInstant(dimensionValue, dateFormat);

Review comment:
       code removed

##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -439,10 +445,31 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
     }
   }
 
+  private static long createTimeInstant(String dimensionValue, String dateFormat) {
+    String updatedDim = dimensionValue;
+    if (!dimensionValue.trim().contains(" ")) {
+      updatedDim += " 00:00:00";

Review comment:
       code removed

##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneIndexExample.scala
##########
@@ -30,7 +30,7 @@ object LuceneIndexExample {
 
   def main(args: Array[String]) {
     val spark = ExampleUtils.createSparkSession("LuceneIndexExample")
-    exampleBody(spark)
+      exampleBody(spark)

Review comment:
       done

##########
File path: core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
##########
@@ -2644,5 +2644,4 @@ private CarbonCommonConstants() {
   public static final String CARBON_MAP_ORDER_PUSHDOWN = "carbon.mapOrderPushDown";
 
   public static final String CARBON_SDK_EMPTY_METADATA_PATH = "emptyMetadataFolder";
-

Review comment:
       done

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,734 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}

Review comment:
       corrected

##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/StructuredStreamingExample.scala
##########
@@ -151,7 +151,7 @@ object StructuredStreamingExample {
           // Write data from socket stream to carbondata file
           qry = readSocketDF.writeStream
             .format("carbondata")
-            .trigger(ProcessingTime("5 seconds"))
+            .trigger(CarbonToSparkAdapter.getProcessingTime("5 seconds"))

Review comment:
       added to scala package from spark_version

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/load/DataLoadProcessorStepOnSpark.scala
##########
@@ -183,7 +183,7 @@ object DataLoadProcessorStepOnSpark {
     val rowConverter = new RowConverterImpl(conf.getDataFields, conf, badRecordLogger)
     rowConverter.initialize()
 
-    TaskContext.get().addTaskCompletionListener { context =>
+    CarbonToSparkAdapter.addTaskCompletionListener {

Review comment:
       addTaskCompletionListener API change in Spark3.1

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateDataSourceTableCommand.scala
##########
@@ -78,10 +78,8 @@ case class CarbonCreateDataSourceTableCommand(
       catalogTable.partitionColumnNames,
       caseSensitiveAnalysis)
     val rows = try {
-      CreateDataSourceTableCommand(
-        catalogTable,
-        ignoreIfExists
-      ).run(sparkSession)
+      org.apache.spark.sql.execution.CreateDataSourceTableCommand
+        .createDataSource(catalogTable, ignoreIfExists, sparkSession)

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
##########
@@ -501,40 +321,6 @@ object CarbonSparkSqlParserUtil {
     }
   }
 
-  /**
-   * Validates the partition columns and return's A tuple of partition columns and partitioner
-   * fields.
-   *
-   * @param partitionColumns        An instance of ColTypeListContext having parser rules for
-   *                                column.
-   * @param colNames                <Seq[String]> Sequence of Table column names.
-   * @param tableProperties         <Map[String, String]> Table property map.
-   * @param partitionByStructFields Seq[StructField] Sequence of partition fields.
-   * @return <Seq[PartitionerField]> A Seq of partitioner fields.
-   */
-  def validatePartitionFields(

Review comment:
       added to common package

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/jobs/SparkBlockletIndexLoaderJob.scala
##########
@@ -160,9 +160,7 @@ class IndexLoaderRDD(
     val reader = indexFormat.createRecordReader(inputSplit, attemptContext)
     val iter = new Iterator[(TableBlockIndexUniqueIdentifier, BlockletIndexDetailsWithSchema)] {
       // in case of success, failure or cancellation clear memory and stop execution
-      context.addTaskCompletionListener { _ =>
-        reader.close()
-      }
+      CarbonToSparkAdapter.addTaskCompletionListener(reader.close())

Review comment:
       addTaskCompletionListener API changed in Spark 3.1

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,735 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener, SessionCatalog}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, ExprId, Expression, ExpressionSet, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{JoinType, QueryPlan, logical}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.trees.{CurrentOrigin, TreeNode}
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, SQLExecution, ShuffledRowRDD, SimpleMode, SparkPlan, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, RefreshTableCommand, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FilePartition, PartitionedFile}
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.metric.SQLShuffleWriteMetricsReporter
+import org.apache.spark.sql.execution.strategy.CarbonDataSourceScan
+import org.apache.spark.sql.hive.HiveExternalCatalog
+import org.apache.spark.sql.internal.{SessionState, SharedState}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.parser.CarbonSparkSqlParserUtil.{checkIfDuplicateColumnExists, convertDbNameToLowerCase, validateStreamingProperty}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.streaming.Trigger
+import org.apache.spark.sql.types.{AbstractDataType, CharType, DataType, Metadata, StringType, StructField, VarcharType}
+import org.apache.spark.sql.util.SparkSQLUtil
+import org.apache.spark.unsafe.types.UTF8String
+
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.SchemaReader
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+import org.apache.carbondata.mv.plans.modular.{GroupBy, ModularPlan, Select}
+import org.apache.carbondata.spark.CarbonOption
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+object CarbonToSparkAdapter {
+
+  def addSparkSessionListener(sparkSession: SparkSession): Unit = {
+    sparkSession.sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        CarbonEnv.carbonEnvMap.remove(sparkSession)
+        ThreadLocalSessionInfo.unsetAll()
+      }
+    })
+  }
+
+  def addSparkListener(sparkContext: SparkContext): Unit = {
+    sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        SparkSession.setDefaultSession(null)
+      }
+    })
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Option[String],
+      attrRef : NamedExpression = null): AttributeReference = {
+    val qf = if (qualifier.nonEmpty) Seq(qualifier.get) else Seq.empty
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qf)
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Seq[String]): AttributeReference = {
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qualifier)
+  }
+
+  def lowerCaseAttribute(expression: Expression): Expression = expression.transform {
+    case attr: AttributeReference =>
+      CarbonToSparkAdapter.createAttributeReference(
+        attr.name.toLowerCase,
+        attr.dataType,
+        attr.nullable,
+        attr.metadata,
+        attr.exprId,
+        attr.qualifier)
+  }
+
+  def createAttributeReference(attr: AttributeReference,
+      attrName: String,
+      newSubsume: String): AttributeReference = {
+    AttributeReference(attrName, attr.dataType)(
+      exprId = attr.exprId,
+      qualifier = newSubsume.split("\n").map(_.trim))
+  }
+
+  def createScalaUDF(s: ScalaUDF, reference: AttributeReference): ScalaUDF = {
+    s.copy(children = Seq(reference))
+  }
+
+  def createExprCode(code: String, isNull: String, value: String, dataType: DataType): ExprCode = {
+    ExprCode(
+      code"$code",
+      JavaCode.isNullVariable(isNull),
+      JavaCode.variable(value, dataType))
+  }
+
+  def createAliasRef(
+      child: Expression,
+      name: String,
+      exprId: ExprId = NamedExpression.newExprId,
+      qualifier: Seq[String] = Seq.empty,
+      explicitMetadata: Option[Metadata] = None,
+      namedExpr: Option[NamedExpression] = None) : Alias = {

Review comment:
       done

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/avro/AvroFileFormatFactory.scala
##########
@@ -0,0 +1,48 @@
+/*
+ * 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.spark.sql.avro

Review comment:
       done

##########
File path: mv/plan/src/main/spark2.3/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala
##########
@@ -47,4 +55,110 @@ object ExpressionHelper {
     reference.qualifier.head
   }
 
+  def getStatisticsObj(outputList: Seq[NamedExpression],
+      plan: LogicalPlan, stats: Statistics,
+      aliasMap: Option[AttributeMap[Attribute]] = None): Statistics = {
+    val output = outputList.map(_.toAttribute)
+    val mapSeq = plan.collect { case n: logical.LeafNode => n }.map {
+      table => AttributeMap(table.output.zip(output))
+    }
+    val rewrites = mapSeq.head
+    val attributes: AttributeMap[ColumnStat] = stats.attributeStats
+    var attributeStats = AttributeMap(attributes.iterator
+      .map { pair => (rewrites(pair._1), pair._2) }.toSeq)
+    if (aliasMap.isDefined) {
+      attributeStats = AttributeMap(
+        attributeStats.map(pair => (aliasMap.get(pair._1), pair._2)).toSeq)
+    }
+    Statistics(stats.sizeInBytes, stats.rowCount, attributeStats, stats.hints)
+  }
+
+  def getOptimizedPlan(s: SubqueryExpression): LogicalPlan = {
+    val Subquery(newPlan) = BirdcageOptimizer.execute(Subquery(s.plan))
+    newPlan
+  }
+
+  def normalizeExpressions(r: NamedExpression, attrs: AttributeSeq): NamedExpression = {
+    QueryPlan.normalizeExprId(r, attrs)
+  }
+
+  def attributeMap(rAliasMap: AttributeMap[Attribute]) : AttributeMap[Expression] = {
+    rAliasMap.asInstanceOf[AttributeMap[Expression]]
+  }
+
+  def seqOfRules : Seq[Rule[LogicalPlan]] = {
+    Seq(
+      // Operator push down
+      PushProjectionThroughUnion,
+      ReorderJoin,
+      EliminateOuterJoin,
+      PushPredicateThroughJoin,
+      PushDownPredicate,
+      ColumnPruning,
+      // Operator combine
+      CollapseRepartition,
+      CollapseProject,
+      CollapseWindow,
+      CombineFilters,
+      CombineLimits,
+      CombineUnions,
+      // Constant folding and strength reduction
+      NullPropagation,
+      FoldablePropagation,
+      ConstantFolding,
+      ReorderAssociativeOperator,
+      // No need to apply LikeSimplification rule while creating MV
+      // as modular plan asCompactSql will be set in schema
+      //        LikeSimplification,
+      BooleanSimplification,
+      SimplifyConditionals,
+      RemoveDispensableExpressions,
+      SimplifyBinaryComparison,
+      EliminateSorts,
+      SimplifyCasts,
+      SimplifyCaseConversionExpressions,
+      RewriteCorrelatedScalarSubquery,
+      EliminateSerialization,
+      RemoveRedundantAliases,
+      RemoveRedundantProject)
+  }
+}
+
+trait getVerboseString extends LeafNode {
+}
+
+trait groupByUnaryNode extends UnaryNode {
+}
+
+trait selectModularPlan extends ModularPlan {
+}
+
+trait unionModularPlan extends ModularPlan {
+}
+
+trait oneRowTableLeafNode extends LeafNode {
+}
+
+object MatchJoin {
+  def unapply(plan : LogicalPlan): Option[(LogicalPlan, LogicalPlan, JoinType, Option[Expression],
+    Option[Any])] = {
+    plan match {
+      case j@Join(left, right, joinType, condition) =>
+        val a = Some(left, right, joinType, condition, None)
+        a

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/joins/BroadCastSIFilterPushJoin.scala
##########
@@ -147,13 +152,15 @@ object BroadCastSIFilterPushJoin {
       inputCopy: Array[InternalRow],
       leftKeys: Seq[Expression],
       rightKeys: Seq[Expression],
-      buildSide: BuildSide,
+      buildSide: CarbonBuildSideType,
       isIndexTable: Boolean = false): Unit = {
 
+    val carbonBuildSide = CarbonBuildSide(buildSide)

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/CarbonSIRebuildRDD.scala
##########
@@ -214,9 +214,7 @@ class CarbonSIRebuildRDD[K, V](
           new SparkDataTypeConverterImpl)
 
         // add task completion listener to clean up the resources
-        context.addTaskCompletionListener { _ =>
-          close()
-        }
+        CarbonToSparkAdapter.addTaskCompletionListener(close())

Review comment:
       addTaskCompletionListener API changed in Spark 3.1

##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -185,8 +224,390 @@ object CarbonToSparkAdapter {
   def getHiveExternalCatalog(sparkSession: SparkSession): HiveExternalCatalog = {
     sparkSession.sessionState.catalog.externalCatalog.asInstanceOf[HiveExternalCatalog]
   }
+
+  def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]) = {
+    FilePartition(index, files.toArray.toSeq)
+  }
+
+  def stringToTimestamp(timestamp: String): Option[Long] = {
+    DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp))
+  }
+
+  def getTableIdentifier(u: UnresolvedRelation): Some[TableIdentifier] = {
+    Some(u.tableIdentifier)
+  }
+
+  def dateToString(date: Int): String = {
+    DateTimeUtils.dateToString(date.toString.toInt)
+  }
+
+  def timeStampToString(timeStamp: Long): String = {
+    DateTimeUtils.timestampToString(timeStamp)
+  }

Review comment:
       done, added to common2.3and2.4

##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -185,8 +224,390 @@ object CarbonToSparkAdapter {
   def getHiveExternalCatalog(sparkSession: SparkSession): HiveExternalCatalog = {
     sparkSession.sessionState.catalog.externalCatalog.asInstanceOf[HiveExternalCatalog]
   }
+
+  def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]) = {
+    FilePartition(index, files.toArray.toSeq)
+  }
+
+  def stringToTimestamp(timestamp: String): Option[Long] = {
+    DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp))
+  }
+
+  def getTableIdentifier(u: UnresolvedRelation): Some[TableIdentifier] = {
+    Some(u.tableIdentifier)
+  }
+
+  def dateToString(date: Int): String = {
+    DateTimeUtils.dateToString(date.toString.toInt)
+  }
+
+  def timeStampToString(timeStamp: Long): String = {
+    DateTimeUtils.timestampToString(timeStamp)
+  }
+
+  def stringToTime(value: String): java.util.Date = {
+    DateTimeUtils.stringToTime(value)
+  }
+
+  def getProcessingTime: String => Trigger = {
+    Trigger.ProcessingTime
+  }
+
+  def addTaskCompletionListener[U](f: => U) {
+    TaskContext.get().addTaskCompletionListener { context =>
+      f
+    }
+  }
+
+  def createShuffledRowRDD(sparkContext: SparkContext, localTopK: RDD[InternalRow],
+      child: SparkPlan, serializer: Serializer): ShuffledRowRDD = {
+    new ShuffledRowRDD(
+      ShuffleExchangeExec.prepareShuffleDependency(
+        localTopK, child.output, SinglePartition, serializer))
+  }
+
+  def getInsertIntoCommand(table: LogicalPlan,
+      partition: Map[String, Option[String]],
+      query: LogicalPlan,
+      overwrite: Boolean,
+      ifPartitionNotExists: Boolean): InsertIntoTable = {
+    InsertIntoTable(
+      table,
+      partition,
+      query,
+      overwrite,
+      ifPartitionNotExists)
+  }
+
+  def getExplainCommandObj(logicalPlan: LogicalPlan = OneRowRelation(),
+      mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(logicalPlan, mode.isDefined)
+  }
+
+  def getExplainCommandObj(mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(OneRowRelation(), mode.isDefined)
+  }
+
+  def invokeAnalyzerExecute(analyzer: Analyzer,
+      plan: LogicalPlan): LogicalPlan = {
+    analyzer.executeAndCheck(plan)
+  }
+
+  def normalizeExpressions(r: NamedExpression, attrs: AttributeSeq): NamedExpression = {
+    QueryPlan.normalizeExprId(r, attrs)
+  }
+
+  def getBuildRight: BuildSide = {
+    BuildRight
+  }
+
+  def getBuildLeft: BuildSide = {
+    BuildLeft
+  }
+
+  type CarbonBuildSideType = BuildSide
+  type InsertIntoStatementWrapper = InsertIntoTable
+
+  def withNewExecutionId[T](sparkSession: SparkSession, queryExecution: QueryExecution): T => T = {
+    SQLExecution.withNewExecutionId(sparkSession, queryExecution)(_)
+  }
+
+  def getTableIdentifier(parts: TableIdentifier): TableIdentifier = {
+    parts
+  }
+
+  def createJoinNode(child: LogicalPlan,
+      targetTable: LogicalPlan,
+      joinType: JoinType,
+      condition: Option[Expression]): Join = {
+    Join(child, targetTable, joinType, condition)
+  }
+
+  def getPartitionsFromInsert(x: InsertIntoStatementWrapper): Map[String, Option[String]] = {
+    x.partition
+  }
+
+  def getStatisticsObj(outputList: Seq[NamedExpression],

Review comment:
       yes removed

##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala
##########
@@ -17,7 +17,7 @@
 
 package org.apache.spark.sql.hive.execution.command
 
-import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
+import org.apache.spark.sql.{CarbonEnv, CarbonToSparkAdapter, Row, SparkSession}

Review comment:
       removed

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -229,8 +270,388 @@ object CarbonToSparkAdapter {
       .unwrapped
       .asInstanceOf[HiveExternalCatalog]
   }
+
+  def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]): FilePartition = {
+    FilePartition(index, files.toArray)
+  }
+
+  def stringToTimestamp(timestamp: String): Option[Long] = {
+    DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp))
+  }
+
+  def stringToTime(value: String): java.util.Date = {
+    DateTimeUtils.stringToTime(value)
+  }
+
+  def timeStampToString(timeStamp: Long): String = {
+    DateTimeUtils.timestampToString(timeStamp)
+  }
+
+  def getTableIdentifier(u: UnresolvedRelation): Some[TableIdentifier] = {
+    Some(u.tableIdentifier)
+  }
+
+  def dateToString(date: Int): String = {
+    DateTimeUtils.dateToString(date.toString.toInt)
+  }
+
+  def getProcessingTime: String => Trigger = {
+    Trigger.ProcessingTime
+  }
+
+  def addTaskCompletionListener[U](f: => U) {
+    TaskContext.get().addTaskCompletionListener { context =>
+      f
+    }
+  }
+
+  def createShuffledRowRDD(sparkContext: SparkContext, localTopK: RDD[InternalRow],
+      child: SparkPlan, serializer: Serializer): ShuffledRowRDD = {
+    new ShuffledRowRDD(
+      ShuffleExchangeExec.prepareShuffleDependency(
+        localTopK, child.output, SinglePartition, serializer))
+  }
+
+  def getInsertIntoCommand(table: LogicalPlan,
+      partition: Map[String, Option[String]],
+      query: LogicalPlan,
+      overwrite: Boolean,
+      ifPartitionNotExists: Boolean): InsertIntoTable = {
+    InsertIntoTable(
+      table,
+      partition,
+      query,
+      overwrite,
+      ifPartitionNotExists)
+  }
+
+  def getExplainCommandObj(logicalPlan: LogicalPlan = OneRowRelation(),
+      mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(logicalPlan, mode.isDefined)
+  }
+
+  def invokeAnalyzerExecute(analyzer: Analyzer,
+      plan: LogicalPlan): LogicalPlan = {
+    analyzer.executeAndCheck(plan)
+  }
+
+  def normalizeExpressions(r: NamedExpression, attrs: AttributeSeq): NamedExpression = {
+    QueryPlan.normalizeExprId(r, attrs)
+  }
+
+  def getBuildRight: BuildSide = {
+    BuildRight
+  }
+
+  def getBuildLeft: BuildSide = {
+    BuildLeft
+  }
+
+  type CarbonBuildSideType = BuildSide
+  type InsertIntoStatementWrapper = InsertIntoTable
+
+  def withNewExecutionId[T](sparkSession: SparkSession, queryExecution: QueryExecution): T => T = {
+    SQLExecution.withNewExecutionId(sparkSession, queryExecution)(_)
+  }
+
+  def createJoinNode(child: LogicalPlan,
+      targetTable: LogicalPlan,
+      joinType: JoinType,
+      condition: Option[Expression]): Join = {
+    Join(child, targetTable, joinType, condition)
+  }
+
+  def getPartitionsFromInsert(x: InsertIntoStatementWrapper): Map[String, Option[String]] = {
+    x.partition
+  }
+
+  def getTableIdentifier(parts: TableIdentifier): TableIdentifier = {
+    parts
+  }
+
+  def getStatisticsObj(outputList: Seq[NamedExpression],
+      plan: LogicalPlan, stats: Statistics,
+      aliasMap: Option[AttributeMap[Attribute]] = None): Statistics = {
+    val output = outputList.map(_.toAttribute)
+    val mapSeq = plan.collect { case n: logical.LeafNode => n }.map {
+      table => AttributeMap(table.output.zip(output))
+    }
+    val rewrites = mapSeq.head
+    val attributes: AttributeMap[ColumnStat] = stats.attributeStats
+    var attributeStats = AttributeMap(attributes.iterator
+      .map { pair => (rewrites(pair._1), pair._2) }.toSeq)
+    if (aliasMap.isDefined) {
+      attributeStats = AttributeMap(
+        attributeStats.map(pair => (aliasMap.get(pair._1), pair._2)).toSeq)
+    }
+    Statistics(stats.sizeInBytes, stats.rowCount, attributeStats, stats.hints)
+  }
+
+  def createRefreshTableCommand(tableIdentifier: TableIdentifier): RefreshTable = {
+    RefreshTable(tableIdentifier)
+  }
+
+  type RefreshTables = RefreshTable

Review comment:
       done

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/execution/CarbonCodegenSupport.scala
##########
@@ -0,0 +1,23 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.spark.sql.execution.joins.HashJoin
+
+trait CarbonCodegenSupport extends SparkPlan with HashJoin {

Review comment:
       done

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
##########
@@ -0,0 +1,182 @@
+/*
+ * 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.spark.sql.execution.strategy
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.CarbonInputMetrics
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.CarbonDatasourceHadoopRelation
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonSparkDataSourceUtil
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, SortOrder, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.{Expression => SparkExpression}
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
+import org.apache.spark.sql.execution.{ColumnarBatchScan, DataSourceScanExec, WholeStageCodegenExec}
+import org.apache.spark.sql.optimizer.CarbonFilters
+import org.apache.spark.sql.types.AtomicType
+
+import org.apache.carbondata.core.index.IndexFilter
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.metadata.schema.BucketingInfo
+import org.apache.carbondata.core.readcommitter.ReadCommittedScope
+import org.apache.carbondata.core.scan.expression.Expression
+import org.apache.carbondata.core.scan.expression.logical.AndExpression
+import org.apache.carbondata.hadoop.CarbonProjection
+import org.apache.carbondata.spark.rdd.CarbonScanRDD
+
+/**
+ *  Physical plan node for scanning data. It is applied for both tables
+ *  USING carbondata and STORED AS carbondata.
+ */
+case class CarbonDataSourceScan(

Review comment:
       done

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/CarbonAnalyzer.scala
##########
@@ -0,0 +1,50 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.analysis.Analyzer
+import org.apache.spark.sql.catalyst.catalog.SessionCatalog
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.util.CarbonReflectionUtils
+
+class CarbonAnalyzer(catalog: SessionCatalog,

Review comment:
       done

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.parser.ParserUtils.string
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{AddTableColumnsContext, CreateHiveTableContext}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkSqlAstBuilder
+import org.apache.spark.sql.internal.SQLConf

Review comment:
       done

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.CarbonToSparkAdapter

Review comment:
       done

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala
##########
@@ -0,0 +1,46 @@
+/*
+ * 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.spark.sql.hive.execution.command
+
+import org.apache.spark.sql.{CarbonEnv, CarbonToSparkAdapter, Row, SparkSession}

Review comment:
       done

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.spark.sql.parser
+

Review comment:
       done

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.
+ */

Review comment:
       done

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonBoundReference.scala
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.spark.sql
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, ExprId, LeafExpression, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.types.DataType
+
+import org.apache.carbondata.core.scan.expression.ColumnExpression
+
+case class CarbonBoundReference(colExp: ColumnExpression, dataType: DataType, nullable: Boolean)

Review comment:
       common in 2.3 and 2.4, added to common code

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkSqlAdapter.scala
##########
@@ -0,0 +1,49 @@
+/*
+ * 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.spark.sql
+
+import org.apache.spark.sql.catalyst.TableIdentifier
+import org.apache.spark.sql.catalyst.expressions.{Attribute, EmptyRow, Expression}
+import org.apache.spark.sql.execution.FileSourceScanExec
+import org.apache.spark.sql.execution.datasources.HadoopFsRelation
+import org.apache.spark.sql.types.StructType
+
+object SparkSqlAdapter {
+
+  def initSparkSQL(): Unit = {
+  }
+
+  def getScanForSegments(
+      @transient relation: HadoopFsRelation,

Review comment:
        FileSourceScanExec API is different in spark2.3, 2.4 and 3.1. Different number of arguments in all 3.

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/SparkSqlAstBuilderWrapper.scala
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.spark.sql.parser
+
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
+import org.apache.spark.sql.execution.SparkSqlAstBuilder
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * use this wrapper to adapter multiple spark versions
+ */
+abstract class SparkSqlAstBuilderWrapper(conf: SQLConf) extends SparkSqlAstBuilder {

Review comment:
       done

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.spark.sql.parser
+
+import org.apache.spark.sql.{CarbonEnv, CarbonThreadUtil, SparkSession}
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkSqlParser
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.CarbonException
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * parser order: carbon parser => spark parser
+ */
+class CarbonExtensionSqlParser(

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/DeleteExecution.scala
##########
@@ -321,7 +321,7 @@ object DeleteExecution {
           deleteStatus = SegmentStatus.SUCCESS
         } catch {
           case e : MultipleMatchingException =>
-            LOGGER.error(e.getMessage)
+          LOGGER.error(e.getMessage)

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/util/CreateTableCommonUtil.scala
##########
@@ -0,0 +1,91 @@
+/*
+ * 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.spark.sql.util
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogUtils}
+import org.apache.spark.sql.execution.datasources.{DataSource, HadoopFsRelation}
+import org.apache.spark.sql.internal.SessionState
+import org.apache.spark.sql.sources.BaseRelation
+import org.apache.spark.sql.types.StructType
+
+object CreateTableCommonUtil {
+
+  def getNewTable(sparkSession: SparkSession, sessionState: SessionState,

Review comment:
       done

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
##########
@@ -360,9 +359,14 @@ object CarbonStore {
 
   private def validateTimeFormat(timestamp: String): Long = {
     try {
-      DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp)).get
+      CarbonToSparkAdapter.stringToTimestamp(timestamp) match {
+        case Some(value) => value
+        case _ =>
+          val errorMessage = "Error: Invalid load start time format: " + timestamp

Review comment:
       done

##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/execution/CarbonCodegenSupport.scala
##########
@@ -14,15 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.spark.sql.execution
 
-package org.apache.carbondata.spark.adapter
+import org.apache.spark.sql.execution.joins.HashJoin
 
-import scala.collection.mutable.ArrayBuffer
+trait CarbonCodegenSupport extends SparkPlan with HashJoin {

Review comment:
       removed hashjoin from BroadCastSIfilterPushjoin

##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+
+object CreateDataSourceTableCommand {

Review comment:
       added to common code 

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/CarbonBoundReference.scala
##########
@@ -42,4 +42,4 @@ case class CarbonBoundReference(colExp: ColumnExpression, dataType: DataType, nu
   override def qualifier: Seq[String] = null
 
   override def newInstance(): NamedExpression = throw new UnsupportedOperationException
-}
\ No newline at end of file
+}

Review comment:
       done

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, CatalogUtils}
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.util.CreateTableCommonUtil.getNewTable
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+
+case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boolean)
+  extends RunnableCommand {
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    assert(table.tableType != CatalogTableType.VIEW)
+    assert(table.provider.isDefined)
+    val sessionState = sparkSession.sessionState
+    if (sessionState.catalog.tableExists(table.identifier)) {
+      if (ignoreIfExists) {
+        return Seq.empty[Row]
+      } else {
+        throw new AnalysisException(s"Table ${table.identifier.unquotedString} already exists.")
+      }
+    }
+    val newTable: CatalogTable = getNewTable(sparkSession, sessionState, table, LOGGER)
+
+    // We will return Nil or throw exception at the beginning if the table already exists, so when
+    // we reach here, the table should not exist and we should set `ignoreIfExists` to false.
+    sessionState.catalog.createTable(newTable, ignoreIfExists = false, validateLocation = false)
+    Seq.empty[Row]
+  }
+}
+
+object CreateDataSourceTableCommand {

Review comment:
       It is common to 3.1, combined it with 3.1

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/DMLStrategy.scala
##########
@@ -240,6 +235,21 @@ object DMLStrategy extends SparkStrategy {
     condition.get.asInstanceOf[ScalaUDF].function.isInstanceOf[InPolygonJoinRangeListUDF]
   }
 
+  object CarbonExtractEquiJoinKeys {
+    def unapply(plan: LogicalPlan): Option[(JoinType, Seq[Expression], Seq[Expression],
+      Option[Expression], LogicalPlan, LogicalPlan)] = {
+      plan match {
+        case join: Join =>
+          ExtractEquiJoinKeys.unapply(join) match {
+              // ignoring hints as carbon is not using them right now

Review comment:
       done

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.spark.sql.hive
+
+import java.util
+
+import org.apache.spark.sql.CarbonToSparkAdapter
+
+import scala.collection.JavaConverters._
+import org.apache.spark.sql.catalyst.CarbonParserUtil
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{AddTableColumnsContext, CreateTableContext, HiveChangeColumnContext}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, QualifiedColType}
+import org.apache.spark.sql.execution.SparkSqlAstBuilder
+import org.apache.spark.sql.execution.command.{AlterTableAddColumnsModel, AlterTableDataTypeChangeModel}
+import org.apache.spark.sql.execution.command.schema.{CarbonAlterTableAddColumnCommand, CarbonAlterTableColRenameDataTypeChangeCommand}
+import org.apache.spark.sql.execution.command.table.CarbonExplainCommand
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.types.{DecimalType, StructField}
+
+trait SqlAstBuilderHelper extends SparkSqlAstBuilder {
+
+

Review comment:
       done

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
##########
@@ -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.spark.sql.execution.strategy
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.CarbonInputMetrics
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonToSparkAdapter}
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonSparkDataSourceUtil
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, SortOrder, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.{Expression => SparkExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
+import org.apache.spark.sql.execution.{DataSourceScanExec, WholeStageCodegenExec}
+import org.apache.spark.sql.execution.metric.SQLMetrics
+import org.apache.spark.sql.optimizer.CarbonFilters
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+import org.apache.carbondata.core.index.IndexFilter
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.metadata.schema.BucketingInfo
+import org.apache.carbondata.core.readcommitter.ReadCommittedScope
+import org.apache.carbondata.core.scan.expression.Expression
+import org.apache.carbondata.core.scan.expression.logical.AndExpression
+import org.apache.carbondata.hadoop.CarbonProjection
+import org.apache.carbondata.spark.rdd.CarbonScanRDD
+
+/**
+ *  Physical plan node for scanning data. It is applied for both tables
+ *  USING carbondata and STORED AS carbondata.
+ */
+case class CarbonDataSourceScan(
+    @transient relation: CarbonDatasourceHadoopRelation,
+    output: Seq[Attribute],
+    partitionFilters: Seq[SparkExpression],
+    dataFilters: Seq[SparkExpression],
+    @transient readComittedScope: ReadCommittedScope,
+    @transient pushedDownProjection: CarbonProjection,
+    @transient pushedDownFilters: Seq[Expression],
+    directScanSupport: Boolean,
+    @transient extraRDD: Option[(RDD[InternalRow], Boolean)] = None,
+    tableIdentifier: Option[TableIdentifier] = None,
+    segmentIds: Option[String] = None)
+  extends DataSourceScanExec {
+
+  override lazy val supportsColumnar: Boolean = CarbonPlanHelper
+    .supportBatchedDataSource(sqlContext, output, extraRDD)
+
+  override protected def doExecuteColumnar(): RDD[ColumnarBatch] = {
+    val numOutputRows = longMetric("numOutputRows")
+    inputRDD.asInstanceOf[RDD[ColumnarBatch]].mapPartitionsInternal { batches =>
+      new Iterator[ColumnarBatch] {
+
+        override def hasNext: Boolean = {
+          val res = batches.hasNext
+          res
+        }
+
+        override def next(): ColumnarBatch = {
+          val batch = batches.next()
+          numOutputRows += batch.numRows()
+          batch
+        }
+      }
+    }
+  }
+
+  override lazy val metrics = Map(
+    "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"))
+
+  lazy val needsUnsafeRowConversion: Boolean = { true }
+
+  override lazy val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) = {
+    val info: BucketingInfo = relation.carbonTable.getBucketingInfo
+    if (info != null) {
+      val cols = info.getListOfColumns.asScala
+      val numBuckets = info.getNumOfRanges
+      val bucketColumns = cols.flatMap { n =>
+        val attrRef = output.find(_.name.equalsIgnoreCase(n.getColumnName))
+        attrRef match {
+          case Some(attr: AttributeReference) =>
+            Some(AttributeReference(attr.name,
+              CarbonSparkDataSourceUtil.convertCarbonToSparkDataType(n.getDataType),
+              attr.nullable,
+              attr.metadata)(attr.exprId, attr.qualifier))
+          case _ => None
+        }
+      }
+      if (bucketColumns.size == cols.size) {
+        // use HashPartitioning will not shuffle
+        (HashPartitioning(bucketColumns, numBuckets), Nil)
+      } else {
+        (UnknownPartitioning(0), Nil)
+      }
+    } else {
+      (UnknownPartitioning(0), Nil)
+    }
+  }
+
+  override lazy val metadata: Map[String, String] = {
+    def seqToString(seq: Seq[Any]) = seq.mkString("[", ", ", "]")
+    val metadata =
+      Map(
+        "ReadSchema" -> seqToString(pushedDownProjection.getAllColumns),
+        "Batched" -> supportsColumnar.toString,
+        "DirectScan" -> (supportsColumnar && directScanSupport).toString,
+        "PushedFilters" -> seqToString(pushedDownFilters.map(_.getStatement)))
+    if (relation.carbonTable.isHivePartitionTable) {
+      metadata + ("PartitionFilters" -> seqToString(partitionFilters)) +
+        ("PartitionCount" -> selectedPartitions.size.toString)
+    } else {
+      metadata
+    }
+  }
+
+  @transient private lazy val indexFilter: IndexFilter = {
+    val filter = pushedDownFilters.reduceOption(new AndExpression(_, _))
+      .map(new IndexFilter(relation.carbonTable, _, true)).orNull
+    if (filter != null && pushedDownFilters.length == 1) {
+      // push down the limit if only one filter
+      filter.setLimit(relation.limit)
+    }
+    filter
+  }
+
+  @transient private lazy val selectedPartitions: Seq[PartitionSpec] = {
+    CarbonFilters
+      .getPartitions(partitionFilters, relation.sparkSession, relation.carbonTable)
+      .orNull
+  }
+
+  private lazy val inputRDD: RDD[InternalRow] = {
+    val carbonRdd = new CarbonScanRDD[InternalRow](
+      relation.sparkSession,
+      pushedDownProjection,
+      indexFilter,
+      relation.identifier,
+      relation.carbonTable.getTableInfo.serialize(),
+      relation.carbonTable.getTableInfo,
+      new CarbonInputMetrics,
+      selectedPartitions,
+      segmentIds = segmentIds)
+    carbonRdd.setVectorReaderSupport(supportsColumnar)
+    carbonRdd.setDirectScanSupport(supportsColumnar && directScanSupport)
+    extraRDD.map(_._1.union(carbonRdd)).getOrElse(carbonRdd)
+  }
+

Review comment:
       done

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.spark.sql.hive
+
+import java.util.concurrent.Callable
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition, ExternalCatalogWithListener, FunctionResourceLoader, GlobalTempViewManager}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.execution.strategy.{CarbonSourceStrategy, DDLStrategy, DMLStrategy, StreamingTableStrategy}
+import org.apache.spark.sql.hive.client.HiveClient
+import org.apache.spark.sql.internal.{SessionState, SQLConf}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule}
+import org.apache.spark.sql.parser.CarbonSparkSqlParser
+
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+
+/**
+ * This class will have carbon catalog and refresh the relation from cache if the carbontable in
+ * carbon catalog is not same as cached carbon relation's carbon table
+ *
+ * @param externalCatalog
+ * @param globalTempViewManager
+ * @param sparkSession
+ * @param functionResourceLoader
+ * @param functionRegistry
+ * @param conf
+ * @param hadoopConf
+ */
+class CarbonHiveSessionCatalog(

Review comment:
       different class definition

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala
##########
@@ -0,0 +1,59 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.log4j.Logger
+import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, CatalogUtils}
+import org.apache.spark.sql.execution.command.RunnableCommand
+import org.apache.spark.sql.util.CreateTableCommonUtil.getNewTable
+
+import org.apache.carbondata.common.logging.LogServiceFactory
+
+case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boolean)
+  extends RunnableCommand {
+
+  val LOGGER: Logger = LogServiceFactory.getLogService(this.getClass.getName)
+
+  override def run(sparkSession: SparkSession): Seq[Row] = {
+    assert(table.tableType != CatalogTableType.VIEW)
+    assert(table.provider.isDefined)
+
+    val sessionState = sparkSession.sessionState
+    if (sessionState.catalog.tableExists(table.identifier)) {
+      if (ignoreIfExists) {
+        return Seq.empty[Row]
+      } else {
+        throw new AnalysisException(s"Table ${table.identifier.unquotedString} already exists.")
+      }

Review comment:
       done

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/execution/CarbonCodegenSupport.scala
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
+import org.apache.spark.sql.execution.joins.HashJoin
+
+trait CarbonCodegenSupport extends SparkPlan with HashJoin {
+
+  // TODO: Spark has started supporting Codegen for Join, Carbon needs to implement the same.
+  override def supportCodegen: Boolean = false

Review comment:
       okay




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-855397465


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644572609



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonBoundReference.scala
##########
@@ -0,0 +1,45 @@
+/*
+ * 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.spark.sql
+
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, ExprId, LeafExpression, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenFallback
+import org.apache.spark.sql.types.DataType
+
+import org.apache.carbondata.core.scan.expression.ColumnExpression
+
+case class CarbonBoundReference(colExp: ColumnExpression, dataType: DataType, nullable: Boolean)

Review comment:
       common in 2.3 and 2.4, added to common code




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655298988



##########
File path: integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/CarbonDataSourceScanHelper.scala
##########
@@ -0,0 +1,85 @@
+/*
+ * 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.spark.sql
+
+import org.apache.spark.CarbonInputMetrics
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression => SparkExpression}
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.execution.{ColumnarBatchScan, DataSourceScanExec}
+import org.apache.spark.sql.execution.strategy.CarbonPlanHelper
+import org.apache.spark.sql.optimizer.CarbonFilters
+
+import org.apache.carbondata.core.index.IndexFilter
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.scan.expression.Expression
+import org.apache.carbondata.core.scan.expression.logical.AndExpression
+import org.apache.carbondata.hadoop.CarbonProjection
+import org.apache.carbondata.spark.rdd.CarbonScanRDD
+
+abstract class  CarbonDataSourceScanHelper(relation: CarbonDatasourceHadoopRelation,
+                                           output: Seq[Attribute],
+    partitionFilters: Seq[SparkExpression],
+    pushedDownFilters: Seq[Expression],
+    pushedDownProjection: CarbonProjection,
+    directScanSupport: Boolean,
+    extraRDD: Option[(RDD[InternalRow], Boolean)],
+    segmentIds: Option[String])
+  extends DataSourceScanExec with ColumnarBatchScan {

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-861407311


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644557581



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/rdd/CarbonSIRebuildRDD.scala
##########
@@ -214,9 +214,7 @@ class CarbonSIRebuildRDD[K, V](
           new SparkDataTypeConverterImpl)
 
         // add task completion listener to clean up the resources
-        context.addTaskCompletionListener { _ =>
-          close()
-        }
+        CarbonToSparkAdapter.addTaskCompletionListener(close())

Review comment:
       addTaskCompletionListener API changed in Spark 3.1




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-851536276


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655281093



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/TestIndexCommand.scala
##########
@@ -39,7 +39,9 @@ class TestIndexCommand extends QueryTest with BeforeAndAfterAll {
 
   val newClass = "org.apache.spark.sql.CarbonSource"
 
-  test("test index create: don't support using non-exist class") {
+  // TODO: This issue will be fixed during alter change column support for spark 3.1.1
+  // REFER: https://issues.apache.org/jira/browse/CARBONDATA-4210
+  ignore("test index create: don't support using non-exist class") {

Review comment:
       changed these test cases back, and changed the expected error message




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-854703925


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655297313



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/joins/BroadCastPolygonFilterPushJoin.scala
##########
@@ -47,16 +45,13 @@ case class BroadCastPolygonFilterPushJoin(
     leftKeys: Seq[Expression],
     rightKeys: Seq[Expression],
     joinType: JoinType,
-    buildSide: BuildSide,
     condition: Option[Expression],
     left: SparkPlan,
     right: SparkPlan
-) extends BinaryExecNode with HashJoin {

Review comment:
       methods overridden from HashJoin needed to be removed, so this itself was not needed.




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-852975102


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644564060



##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.
+ */

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644521946



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -439,10 +445,31 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
     }
   }
 
+  private static long createTimeInstant(String dimensionValue, String dateFormat) {
+    String updatedDim = dimensionValue;
+    if (!dimensionValue.trim().contains(" ")) {
+      updatedDim += " 00:00:00";

Review comment:
       code removed




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644562741



##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/execution/CarbonCodegenSupport.scala
##########
@@ -0,0 +1,23 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.spark.sql.execution.joins.HashJoin
+
+trait CarbonCodegenSupport extends SparkPlan with HashJoin {

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-865990752


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-865069353


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r645302121



##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.spark.sql.{Row, SparkSession}
+import org.apache.spark.sql.catalyst.catalog.CatalogTable
+
+object CreateDataSourceTableCommand {

Review comment:
       added to common code 




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

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



[GitHub] [carbondata] vikramahuja1001 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-866627940


   Main Carbondata Jira : CARBONDATA-4190
   Sub-Task Jira: CARBONDATA-4229


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644540782



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonCreateDataSourceTableCommand.scala
##########
@@ -78,10 +78,8 @@ case class CarbonCreateDataSourceTableCommand(
       catalogTable.partitionColumnNames,
       caseSensitiveAnalysis)
     val rows = try {
-      CreateDataSourceTableCommand(
-        catalogTable,
-        ignoreIfExists
-      ).run(sparkSession)
+      org.apache.spark.sql.execution.CreateDataSourceTableCommand
+        .createDataSource(catalogTable, ignoreIfExists, sparkSession)

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-851622588


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


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

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646332185



##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -232,9 +236,9 @@ object CarbonToSparkAdapter {
 }
 
 class CarbonOptimizer(
-    session: SparkSession,
-    catalog: SessionCatalog,
-    optimizer: Optimizer) extends Optimizer(catalog) {
+                       session: SparkSession,

Review comment:
       revert this




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655281152



##########
File path: integration/spark/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
##########
@@ -38,13 +38,13 @@ import org.apache.carbondata.spark.exception.ProcessMetaDataException
 
 class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll {
 
-  var threshold: Int = _
+  var threshold: String = _
 
   override def beforeAll {
 
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
-    threshold = sqlContext.getConf("spark.sql.autoBroadcastJoinThreshold").toInt
+    threshold = sqlContext.getConf("spark.sql.autoBroadcastJoinThreshold")

Review comment:
       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:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-848154207


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


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

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



[GitHub] [carbondata] akashrn5 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655292078



##########
File path: examples/spark/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
##########
@@ -104,7 +104,8 @@ class RunExamples extends QueryTest with BeforeAndAfterAll {
     TableLevelCompactionOptionExample.exampleBody(spark)
   }
 
-  test("LuceneIndexExample") {
+  // Below test case ignored due to the Deadlock in spark code

Review comment:
       add a spark jira if its reported already, and add a TODO here to revert once the spark jira is resolved

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
##########
@@ -135,34 +137,34 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
       var transformed: Boolean = false
       // Create a dummy projection to include filter conditions
       var newPlan: LogicalPlan = null
-      if (table.tableIdentifier.database.isDefined) {
+      if (tableIdentifier.database.isDefined) {
         newPlan = parser.parsePlan("select * from  " +
-           table.tableIdentifier.database.getOrElse("") + "." +
-           table.tableIdentifier.table + " " + alias.getOrElse("") + " " + filter)
+           tableIdentifier.database.getOrElse("") + "." +

Review comment:
       can you replace with POINT in all places in this class?

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
##########
@@ -772,58 +774,62 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     }
 
   def getFields(schema: Seq[StructField], isExternal: Boolean = false): Seq[Field] = {
-    def getScannerInput(col: StructField,
-        columnComment: String,
-        columnName: String) = {
-      if (col.dataType.catalogString == "float" && !isExternal) {
-        '`' + columnName + '`' + " double" + columnComment
+    schema.map { col =>
+      // TODO: Spark has started supporting CharType/VarChar types in Spark 3.1 but both are
+      //  marked as experimental. Adding a hack to change to string for now.

Review comment:
       please mention the jira here in TODO

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/IUDCommonUtil.scala
##########
@@ -44,7 +44,7 @@ object IUDCommonUtil {
       case unresolvedRelation: UnresolvedRelation =>
         val dbAndTb =
           sparkSession.sessionState.catalog.asInstanceOf[HiveSessionCatalog].getCurrentDatabase +
-          "." + unresolvedRelation.tableIdentifier.table
+          "." + CarbonToSparkAdapter.getTableIdentifier(unresolvedRelation).get.table

Review comment:
       replace with constant POINT here

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.spark.sql.parser
+
+import scala.collection.mutable
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.sql.{CarbonThreadUtil, CarbonToSparkAdapter, SparkSession}
+import org.apache.spark.sql.catalyst.parser.{AbstractSqlParser, SqlBaseParser}
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.{SparkSqlAstBuilder, SparkSqlParser}
+import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
+import org.apache.spark.sql.types.StructField
+import org.apache.spark.sql.util.CarbonException
+import org.apache.spark.util.CarbonReflectionUtils
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * Concrete parser for Spark SQL statements and carbon specific
+ * statements
+ */
+class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends SparkSqlParser {
+
+  val parser = new CarbonSpark2SqlParser
+
+  override val astBuilder = CarbonReflectionUtils.getAstBuilder(conf, parser, sparkSession)
+
+  private val substitutor = new VariableSubstitution
+
+  override def parsePlan(sqlText: String): LogicalPlan = {
+    CarbonThreadUtil.updateSessionInfoToCurrentThread(sparkSession)
+    try {
+      val parsedPlan = super.parsePlan(sqlText)
+      CarbonScalaUtil.cleanParserThreadLocals
+      parsedPlan
+    } catch {
+      case ce: MalformedCarbonCommandException =>
+        CarbonScalaUtil.cleanParserThreadLocals
+        throw ce
+      case ex: Throwable =>
+        try {
+          parser.parse(sqlText)
+        } catch {
+          case mce: MalformedCarbonCommandException =>
+            throw mce
+          case e: Throwable =>
+            CarbonException.analysisException(
+              s"""== Parse1 ==
+                 |${ex.getMessage}
+                 |== Parse2 ==
+                 |${e.getMessage}
+               """.stripMargin.trim)
+        }
+    }
+  }
+
+  protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = {
+    super.parse(substitutor.substitute(command))(toResult)
+  }
+}
+
+class CarbonHelperSqlAstBuilder(conf: SQLConf,
+    parser: CarbonSpark2SqlParser,
+    sparkSession: SparkSession)
+  extends SparkSqlAstBuilderWrapper(conf) {
+  /**
+   * Parse a key-value map from a [[TablePropertyListContext]], assuming all values are specified.
+   */
+  override def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = {
+    val props = visitTablePropertyList(ctx)
+    CarbonSparkSqlParserUtil.visitPropertyKeyValues(ctx, props)
+  }
+
+  def getPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String]
+  = {

Review comment:
       move this line above

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,263 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+
+import scala.collection.mutable.ArrayBuffer
+
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSet, Expression, ExprId, NamedExpression, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.Optimizer
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FilePartition, PartitionedFile}
+import org.apache.spark.sql.hive.HiveExternalCatalog
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.types.{DataType, Metadata, StringType}
+
+import org.apache.carbondata.core.util.{CarbonProperties, ThreadLocalSessionInfo}
+import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF}
+
+object CarbonToSparkAdapter extends SparkVersionAdapter {
+
+  def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]): FilePartition = {
+    FilePartition(index, files.toArray)
+  }
+
+  def addSparkSessionListener(sparkSession: SparkSession): Unit = {
+    sparkSession.sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        CarbonEnv.carbonEnvMap.remove(sparkSession)
+        ThreadLocalSessionInfo.unsetAll()
+      }
+    })
+  }
+
+  def addSparkListener(sparkContext: SparkContext): Unit = {
+    sparkContext.addSparkListener(new SparkListener {
+      override def onApplicationEnd(applicationEnd: SparkListenerApplicationEnd): Unit = {
+        SparkSession.setDefaultSession(null)
+      }
+    })
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Option[String],
+      attrRef : NamedExpression = null): AttributeReference = {
+    val qf = if (qualifier.nonEmpty) Seq(qualifier.get) else Seq.empty
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qf)
+  }
+
+  def createAttributeReference(
+      name: String,
+      dataType: DataType,
+      nullable: Boolean,
+      metadata: Metadata,
+      exprId: ExprId,
+      qualifier: Seq[String]): AttributeReference = {
+    AttributeReference(
+      name,
+      dataType,
+      nullable,
+      metadata)(exprId, qualifier)
+  }
+
+  def lowerCaseAttribute(expression: Expression): Expression = expression.transform {
+    case attr: AttributeReference =>
+      CarbonToSparkAdapter.createAttributeReference(
+        attr.name.toLowerCase,
+        attr.dataType,
+        attr.nullable,
+        attr.metadata,
+        attr.exprId,
+        attr.qualifier)
+  }
+
+  def createAttributeReference(attr: AttributeReference,
+      attrName: String,
+      newSubsume: String): AttributeReference = {
+    AttributeReference(attrName, attr.dataType)(
+      exprId = attr.exprId,
+      qualifier = newSubsume.split("\n").map(_.trim))
+  }
+
+  def createScalaUDF(s: ScalaUDF, reference: AttributeReference): ScalaUDF = {
+    s.copy(children = Seq(reference))
+  }
+
+  def createExprCode(code: String, isNull: String, value: String, dataType: DataType): ExprCode = {
+    ExprCode(
+      code"$code",
+      JavaCode.isNullVariable(isNull),
+      JavaCode.variable(value, dataType))
+  }
+
+  def createAliasRef(
+      child: Expression,
+      name: String,
+      exprId: ExprId = NamedExpression.newExprId,
+      qualifier: Seq[String] = Seq.empty,
+      explicitMetadata: Option[Metadata] = None) : Alias = {
+    Alias(child, name)(exprId, qualifier, explicitMetadata)
+  }
+
+  def createAliasRef(
+      child: Expression,
+      name: String,
+      exprId: ExprId,
+      qualifier: Option[String]) : Alias = {
+    Alias(child, name)(exprId,
+      if (qualifier.isEmpty) Seq.empty else Seq(qualifier.get),
+      None)

Review comment:
       move this line above

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
##########
@@ -835,10 +841,21 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         field.dataType = Some("double")
       }
       field.rawSchema = scannerInput
-      if (col.getComment().isDefined) {
+      if (comment.isDefined) {
         field.columnComment = plainComment
       }
       field
+  }
+
+  def getScannerInput(dataType: DataType,
+                      columnComment: String,
+                      columnName: String,
+                      isExternal: Boolean): String = {

Review comment:
       please correct the code style here

##########
File path: mv/plan/src/main/spark2.3/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala
##########
@@ -17,7 +17,7 @@
 
 package org.apache.carbondata.mv.plans.modular
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, ExprId, Expression, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, ExprId, NamedExpression}

Review comment:
       revert this




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

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



[GitHub] [carbondata] vikramahuja1001 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-865721156


   Spark 3.1 result: http://121.244.95.60:12602/job/ApacheCarbon_PR_Builder_3.1_Compile_and_UT_vikram/95/


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647123937



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/cleanfiles/TestCleanFileCommand.scala
##########
@@ -100,6 +100,7 @@ class TestCleanFileCommand extends QueryTest with BeforeAndAfterAll {
   test("clean up table and test trash folder with Marked For Delete and Compacted segments") {
     // do not send MFD folders to trash
     createTable()
+    sql(s"""Show Tables """).show()

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



[GitHub] [carbondata] vikramahuja1001 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-865220414


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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647120455



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
##########
@@ -0,0 +1,154 @@
+/*
+ * 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.spark.sql.parser
+
+import scala.collection.mutable
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.sql.{CarbonThreadUtil, SparkSession, SparkVersionAdapter}
+import org.apache.spark.sql.catalyst.parser.{AbstractSqlParser, SqlBaseParser}
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.{SparkSqlAstBuilder, SparkSqlParser}
+import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
+import org.apache.spark.sql.types.StructField
+import org.apache.spark.sql.util.CarbonException
+import org.apache.spark.util.CarbonReflectionUtils
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * Concrete parser for Spark SQL statements and carbon specific
+ * statements
+ */
+class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends SparkSqlParser {
+
+  val parser = new CarbonSpark2SqlParser
+
+  override val astBuilder = CarbonReflectionUtils.getAstBuilder(conf, parser, sparkSession)
+
+  private val substitutor = new VariableSubstitution
+
+  override def parsePlan(sqlText: String): LogicalPlan = {
+    CarbonThreadUtil.updateSessionInfoToCurrentThread(sparkSession)
+    try {
+      val parsedPlan = super.parsePlan(sqlText)
+      CarbonScalaUtil.cleanParserThreadLocals
+      parsedPlan
+    } catch {
+      case ce: MalformedCarbonCommandException =>
+        CarbonScalaUtil.cleanParserThreadLocals
+        throw ce
+      case ex: Throwable =>
+        try {
+          parser.parse(sqlText)
+        } catch {
+          case mce: MalformedCarbonCommandException =>
+            throw mce
+          case e: Throwable =>
+            CarbonException.analysisException(
+              s"""== Parse1 ==
+                 |${ex.getMessage}
+                 |== Parse2 ==
+                 |${e.getMessage}
+               """.stripMargin.trim)
+        }
+    }
+  }
+
+  protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = {
+    super.parse(substitutor.substitute(command))(toResult)
+  }
+}
+
+class CarbonHelperSqlAstBuilder(conf: SQLConf,
+    parser: CarbonSpark2SqlParser,
+    sparkSession: SparkSession)
+  extends SparkSqlAstBuilderWrapper(conf) {
+  /**
+   * Parse a key-value map from a [[TablePropertyListContext]], assuming all values are specified.
+   */
+  override def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = {
+    val props = visitTablePropertyList(ctx)
+    CarbonSparkSqlParserUtil.visitPropertyKeyValues(ctx, props)
+  }
+
+  def getPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String]
+  = {
+    Option(ctx).map(visitPropertyKeyValues)
+      .getOrElse(Map.empty)
+  }
+
+  def createCarbonTable(createTableTuple: (CreateTableHeaderContext, SkewSpecContext,
+    BucketSpecContext, PartitionFieldListContext, ColTypeListContext, TablePropertyListContext,
+    LocationSpecContext, Option[String], TerminalNode, QueryContext, String)): LogicalPlan = {
+    // val parser = new CarbonSpark2SqlParser

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647101730



##########
File path: mv/plan/src/main/spark2.4/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala
##########
@@ -17,9 +17,11 @@
 
 package org.apache.carbondata.mv.plans.modular
 
-import org.apache.spark.sql.catalyst.expressions.{Alias, AttributeReference, Expression, ExprId, NamedExpression}
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSeq, Expression, ExprId, NamedExpression, SubqueryExpression}

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-854540268


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-855397487


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


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

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646310399



##########
File path: integration/flink/pom.xml
##########
@@ -30,6 +30,11 @@
             <artifactId>carbondata-format</artifactId>
             <version>${project.version}</version>
         </dependency>
+        <dependency>
+            <groupId>com.thoughtworks.paranamer</groupId>

Review comment:
       add LICENSE for this library.




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-864941954


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-856529914


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647101196



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
##########
@@ -544,12 +545,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     }
 
   protected lazy val explainPlan: Parser[LogicalPlan] =
-    (EXPLAIN ~> opt(EXTENDED)) ~ start ^^ {
-      case isExtended ~ logicalPlan =>
+    (EXPLAIN ~> opt(MODE)) ~ start ^^ {
+      case mode ~ logicalPlan =>

Review comment:
       Carbon does not support any other modes at the moment. Have created a JIRA for that. Refer: https://issues.apache.org/jira/browse/CARBONDATA-4201




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647120985



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/addsegment/AddSegmentTestCase.scala
##########
@@ -298,7 +301,7 @@ class AddSegmentTestCase extends QueryTest with BeforeAndAfterAll {
     for (i <- 0 until 10) {
       sql(s"alter table addsegment1 add segment " +
           s"options('path'='${ newPath + i }', 'format'='carbon')").collect()
-
+      sql("select count(*) from addsegment1").show()

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



[GitHub] [carbondata] vikramahuja1001 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-851854059


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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646308564



##########
File path: index/examples/pom.xml
##########
@@ -81,9 +81,6 @@
   <profiles>
     <profile>
       <id>spark-2.3</id>
-      <activation>
-        <activeByDefault>true</activeByDefault>
-      </activation>

Review comment:
       add 3.1 profile




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644577121



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.spark.sql.parser
+
+import org.apache.spark.sql.{CarbonEnv, CarbonThreadUtil, SparkSession}
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkSqlParser
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.CarbonException
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * parser order: carbon parser => spark parser
+ */
+class CarbonExtensionSqlParser(

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



[GitHub] [carbondata] Jeromestein commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
Jeromestein commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-889029698


   > Hi @Jeromestein , you can get in touch with us via the mailing list as well you can also slack to get in touch with the community. [Mailing List](http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/) and [Slack Link](carbondataworkspace.slack.com)
   > You can check the Subtasks of the main [Spark-3.1 Jira](https://issues.apache.org/jira/browse/CARBONDATA-4190) and start working on them.
   
   Thank you very much. I will try.
   
   (Your stack link is not available, showing that "nietsemorej@gmail.com doesn’t have an account on this workspace". Maybe you should send me an invitation link?)


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

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655298384



##########
File path: processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/RowParserImpl.java
##########
@@ -109,7 +109,9 @@ public RowParserImpl(DataField[] output, CarbonDataLoadConfiguration configurati
       return new String[numberOfColumns];
     }
     // If number of columns are less in a row then create new array with same size of header.

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646326285



##########
File path: streaming/src/main/spark2.x/org.apache.carbondata.util/SparkStreamingUtil.scala
##########
@@ -15,14 +15,15 @@
  * limitations under the License.
  */
 
-package org.apache.carbondata.spark.adapter
+package org.apache.carbondata.util
 
-import scala.collection.mutable.ArrayBuffer
+import org.apache.spark.sql.Row
+import org.apache.spark.sql.catalyst.InternalRow
+import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder
 
-import org.apache.spark.sql.execution.datasources.{FilePartition, PartitionedFile}
+object SparkStreamingUtil {

Review comment:
       package structure has "." instead of "/" to seperate packages




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-853242957


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644557146



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/joins/BroadCastSIFilterPushJoin.scala
##########
@@ -147,13 +152,15 @@ object BroadCastSIFilterPushJoin {
       inputCopy: Array[InternalRow],
       leftKeys: Seq[Expression],
       rightKeys: Seq[Expression],
-      buildSide: BuildSide,
+      buildSide: CarbonBuildSideType,
       isIndexTable: Boolean = false): Unit = {
 
+    val carbonBuildSide = CarbonBuildSide(buildSide)

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r645301549



##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/execution/CarbonCodegenSupport.scala
##########
@@ -14,15 +14,10 @@
  * See the License for the specific language governing permissions and
  * limitations under the License.
  */
+package org.apache.spark.sql.execution
 
-package org.apache.carbondata.spark.adapter
+import org.apache.spark.sql.execution.joins.HashJoin
 
-import scala.collection.mutable.ArrayBuffer
+trait CarbonCodegenSupport extends SparkPlan with HashJoin {

Review comment:
       removed hashjoin from BroadCastSIfilterPushjoin




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r645381006



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/execution/CarbonCodegenSupport.scala
##########
@@ -0,0 +1,29 @@
+/*
+ * 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.spark.sql.execution
+
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
+import org.apache.spark.sql.execution.joins.HashJoin
+
+trait CarbonCodegenSupport extends SparkPlan with HashJoin {
+
+  // TODO: Spark has started supporting Codegen for Join, Carbon needs to implement the same.
+  override def supportCodegen: Boolean = false

Review comment:
       okay
   https://issues.apache.org/jira/browse/CARBONDATA-4197

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/command/SICreationCommand.scala
##########
@@ -704,5 +705,13 @@ private[sql] case class CarbonCreateSecondaryIndexCommand(
     columnSchema
   }
 
+  override def clone(): LogicalPlan = {
+    CarbonCreateSecondaryIndexCommand(indexModel,

Review comment:
       this method is called from spark side

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
##########
@@ -604,11 +605,12 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
 
 
   protected lazy val alterTableColumnRenameAndModifyDataType: Parser[LogicalPlan] =
-    ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ CHANGE ~ ident ~ ident ~
-    ident ~ opt("(" ~> rep1sep(valueOptions, ",") <~ ")") <~ opt(";") ^^ {
-      case dbName ~ table ~ change ~ columnName ~ columnNameCopy ~ dataType ~ values =>
+    ALTER ~> TABLE ~> (ident <~ ".").? ~ ident ~ (CHANGE ~> ident) ~ ident ~ ident ~

Review comment:
       it already exists

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
##########
@@ -67,6 +68,8 @@ object DDLStrategy extends SparkStrategy {
       case changeColumn: AlterTableChangeColumnCommand
         if isCarbonTable(changeColumn.tableName) =>
         ExecutedCommandExec(DDLHelper.changeColumn(changeColumn, sparkSession)) :: Nil
+      case changeColumn: CarbonAlterTableColRenameDataTypeChangeCommand =>

Review comment:
       Previously spark was parsing the command, now spark has removed the support. We could either alter the command definition to match with newly updated spark or could parse on our own. Since, the method was already in carbondata code, we decided to parse it in carbondata only.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
##########
@@ -67,6 +68,8 @@ object DDLStrategy extends SparkStrategy {
       case changeColumn: AlterTableChangeColumnCommand
         if isCarbonTable(changeColumn.tableName) =>
         ExecutedCommandExec(DDLHelper.changeColumn(changeColumn, sparkSession)) :: Nil
+      case changeColumn: CarbonAlterTableColRenameDataTypeChangeCommand =>

Review comment:
       Previously spark was parsing the alter column command, now spark has removed the support. We could either alter the command definition to match with newly updated spark or could parse on our own. Since, the method was already in carbondata code, we decided to parse it in carbondata only.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/DDLStrategy.scala
##########
@@ -67,6 +68,8 @@ object DDLStrategy extends SparkStrategy {
       case changeColumn: AlterTableChangeColumnCommand
         if isCarbonTable(changeColumn.tableName) =>
         ExecutedCommandExec(DDLHelper.changeColumn(changeColumn, sparkSession)) :: Nil
+      case changeColumn: CarbonAlterTableColRenameDataTypeChangeCommand =>

Review comment:
       Previously spark was parsing the alter column rename command, now spark has removed the support. We could either alter the command definition to match with newly updated spark or could parse on our own. Since, the method was already in carbondata code, we decided to parse it in carbondata only.

##########
File path: integration/spark/src/main/scala/org/apache/carbondata/spark/util/CarbonSparkUtil.scala
##########
@@ -181,7 +182,7 @@ object CarbonSparkUtil {
    */
   def createHadoopJob(conf: Configuration = FileFactory.getConfiguration): Job = {
     val jobConf = new JobConf(conf)
-    SparkHadoopUtil.get.addCredentials(jobConf)
+    SparkUtil.addCredentials(jobConf)

Review comment:
       spark has made addCredentials method private to spark, can't access it directly from carbondata package




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

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646310548



##########
File path: integration/flink/pom.xml
##########
@@ -193,9 +220,7 @@
     <profiles>
         <profile>
             <id>spark-2.3</id>
-            <activation>
-                <activeByDefault>true</activeByDefault>
-            </activation>
+          <activation><activeByDefault>true</activeByDefault></activation>

Review comment:
       revert this 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:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-854810300


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-851628317


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


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

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646333532



##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -187,11 +193,10 @@ object CarbonToSparkAdapter {
   }
 }
 
-
 class CarbonOptimizer(
-    session: SparkSession,
-    catalog: SessionCatalog,
-    optimizer: Optimizer) extends Optimizer(catalog) {
+                       session: SparkSession,

Review comment:
       revert this 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:
users@infra.apache.org



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-851601038


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655892561



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
##########
@@ -772,58 +774,62 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
     }
 
   def getFields(schema: Seq[StructField], isExternal: Boolean = false): Seq[Field] = {
-    def getScannerInput(col: StructField,
-        columnComment: String,
-        columnName: String) = {
-      if (col.dataType.catalogString == "float" && !isExternal) {
-        '`' + columnName + '`' + " double" + columnComment
+    schema.map { col =>
+      // TODO: Spark has started supporting CharType/VarChar types in Spark 3.1 but both are
+      //  marked as experimental. Adding a hack to change to string for now.

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-853249378


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r645396128



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/test/SparkTestQueryExecutor.scala
##########
@@ -71,6 +71,12 @@ object SparkTestQueryExecutor {
     .config("spark.sql.warehouse.dir", warehouse)
     .config("spark.sql.crossJoin.enabled", "true")
     .config("spark.sql.extensions", extensions)
+    .config("spark.sql.storeAssignmentPolicy", "legacy")
+    .config("spark.sql.legacy.timeParserPolicy", "legacy")

Review comment:
       done

##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/util/ExampleUtils.scala
##########
@@ -93,6 +93,7 @@ object ExampleUtils {
       .config("spark.driver.host", "localhost")
       .config("spark.sql.crossJoin.enabled", "true")
       .config("spark.sql.extensions", "org.apache.spark.sql.CarbonExtensions")
+      .config("spark.sql.legacy.timeParserPolicy", "LEGACY")

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



[GitHub] [carbondata] Indhumathi27 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
Indhumathi27 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646336533



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/carbondata/execution/datasources/SparkCarbonFileFormat.scala
##########
@@ -86,6 +87,11 @@ class SparkCarbonFileFormat extends FileFormat
       options: Map[String, String],
       files: Seq[FileStatus]): Option[StructType] = {
     val conf = sparkSession.sessionState.newHadoopConf()
+    if (options.isEmpty && files.isEmpty) {

Review comment:
       path is removed from options from 3.1, so this change is required to throw the proper exception




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

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



[GitHub] [carbondata] brijoobopanna commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
brijoobopanna commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-865897794


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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644576944



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/SparkSqlAstBuilderWrapper.scala
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.spark.sql.parser
+
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
+import org.apache.spark.sql.execution.SparkSqlAstBuilder
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * use this wrapper to adapter multiple spark versions
+ */
+abstract class SparkSqlAstBuilderWrapper(conf: SQLConf) extends SparkSqlAstBuilder {

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644521888



##########
File path: core/src/main/java/org/apache/carbondata/core/util/DataTypeUtil.java
##########
@@ -439,10 +445,31 @@ public static Object getDataDataTypeForNoDictionaryColumn(String dimensionValue,
     }
   }
 
+  private static long createTimeInstant(String dimensionValue, String dateFormat) {
+    String updatedDim = dimensionValue;
+    if (!dimensionValue.trim().contains(" ")) {
+      updatedDim += " 00:00:00";
+    }
+    Instant instant = Instant.from(ZonedDateTime
+        .of(LocalDateTime.parse(updatedDim, DateTimeFormatter.ofPattern(dateFormat)),
+        ZoneId.systemDefault()));
+    validateTimeStampRange(instant.getEpochSecond());
+    long us = Math.multiplyExact(instant.getEpochSecond(), 1000L);
+    return Math.addExact(us, instant.getNano() * 1000L);
+  }
+
   private static Object parseTimestamp(String dimensionValue, String dateFormat) {
     Date dateToStr;
     DateFormat dateFormatter = null;
     long timeValue;
+    if (Boolean.parseBoolean(CarbonProperties.getInstance().getProperty(CarbonCommonConstants
+        .CARBON_SPARK_VERSION_SPARK3))) {
+      try {
+        return createTimeInstant(dimensionValue, dateFormat);

Review comment:
       code removed




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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-854705223


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655893724



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala
##########
@@ -0,0 +1,153 @@
+/*
+ * 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.spark.sql.parser
+
+import scala.collection.mutable
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.sql.{CarbonThreadUtil, CarbonToSparkAdapter, SparkSession}
+import org.apache.spark.sql.catalyst.parser.{AbstractSqlParser, SqlBaseParser}
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.{SparkSqlAstBuilder, SparkSqlParser}
+import org.apache.spark.sql.internal.{SQLConf, VariableSubstitution}
+import org.apache.spark.sql.types.StructField
+import org.apache.spark.sql.util.CarbonException
+import org.apache.spark.util.CarbonReflectionUtils
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * Concrete parser for Spark SQL statements and carbon specific
+ * statements
+ */
+class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends SparkSqlParser {
+
+  val parser = new CarbonSpark2SqlParser
+
+  override val astBuilder = CarbonReflectionUtils.getAstBuilder(conf, parser, sparkSession)
+
+  private val substitutor = new VariableSubstitution
+
+  override def parsePlan(sqlText: String): LogicalPlan = {
+    CarbonThreadUtil.updateSessionInfoToCurrentThread(sparkSession)
+    try {
+      val parsedPlan = super.parsePlan(sqlText)
+      CarbonScalaUtil.cleanParserThreadLocals
+      parsedPlan
+    } catch {
+      case ce: MalformedCarbonCommandException =>
+        CarbonScalaUtil.cleanParserThreadLocals
+        throw ce
+      case ex: Throwable =>
+        try {
+          parser.parse(sqlText)
+        } catch {
+          case mce: MalformedCarbonCommandException =>
+            throw mce
+          case e: Throwable =>
+            CarbonException.analysisException(
+              s"""== Parse1 ==
+                 |${ex.getMessage}
+                 |== Parse2 ==
+                 |${e.getMessage}
+               """.stripMargin.trim)
+        }
+    }
+  }
+
+  protected override def parse[T](command: String)(toResult: SqlBaseParser => T): T = {
+    super.parse(substitutor.substitute(command))(toResult)
+  }
+}
+
+class CarbonHelperSqlAstBuilder(conf: SQLConf,
+    parser: CarbonSpark2SqlParser,
+    sparkSession: SparkSession)
+  extends SparkSqlAstBuilderWrapper(conf) {
+  /**
+   * Parse a key-value map from a [[TablePropertyListContext]], assuming all values are specified.
+   */
+  override def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = {
+    val props = visitTablePropertyList(ctx)
+    CarbonSparkSqlParserUtil.visitPropertyKeyValues(ctx, props)
+  }
+
+  def getPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String]
+  = {

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655126999



##########
File path: integration/spark/src/test/scala/org/apache/spark/carbondata/bucketing/TableBucketingTestCase.scala
##########
@@ -38,13 +38,13 @@ import org.apache.carbondata.spark.exception.ProcessMetaDataException
 
 class TableBucketingTestCase extends QueryTest with BeforeAndAfterAll {
 
-  var threshold: Int = _
+  var threshold: String = _
 
   override def beforeAll {
 
     CarbonProperties.getInstance()
       .addProperty(CarbonCommonConstants.CARBON_TIMESTAMP_FORMAT, "yyyy/MM/dd")
-    threshold = sqlContext.getConf("spark.sql.autoBroadcastJoinThreshold").toInt
+    threshold = sqlContext.getConf("spark.sql.autoBroadcastJoinThreshold")

Review comment:
       use SQLConf.AUTO_BROADCASTJOIN_THRESHOLD




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644537189



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,734 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}

Review comment:
       corrected




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

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655123704



##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/TestIndexCommand.scala
##########
@@ -39,7 +39,9 @@ class TestIndexCommand extends QueryTest with BeforeAndAfterAll {
 
   val newClass = "org.apache.spark.sql.CarbonSource"
 
-  test("test index create: don't support using non-exist class") {
+  // TODO: This issue will be fixed during alter change column support for spark 3.1.1
+  // REFER: https://issues.apache.org/jira/browse/CARBONDATA-4210
+  ignore("test index create: don't support using non-exist class") {

Review comment:
       why are these tests commented?

##########
File path: integration/spark/src/test/scala/org/apache/carbondata/spark/testsuite/index/TestIndexCommand.scala
##########
@@ -39,7 +39,9 @@ class TestIndexCommand extends QueryTest with BeforeAndAfterAll {
 
   val newClass = "org.apache.spark.sql.CarbonSource"
 
-  test("test index create: don't support using non-exist class") {
+  // TODO: This issue will be fixed during alter change column support for spark 3.1.1
+  // REFER: https://issues.apache.org/jira/browse/CARBONDATA-4210
+  ignore("test index create: don't support using non-exist class") {

Review comment:
       why are these tests ignored?




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

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



[GitHub] [carbondata] akashrn5 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-866504108


   LGTM


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

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



[GitHub] [carbondata] akashrn5 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
akashrn5 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r639690258



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
##########
@@ -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.spark.sql.execution.strategy
+
+import scala.collection.JavaConverters._
+
+import org.apache.spark.CarbonInputMetrics
+import org.apache.spark.rdd.RDD
+import org.apache.spark.sql.{CarbonDatasourceHadoopRelation, CarbonToSparkAdapter}
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonSparkDataSourceUtil
+import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeReference, SortOrder, UnsafeProjection}
+import org.apache.spark.sql.catalyst.expressions.{Expression => SparkExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext
+import org.apache.spark.sql.catalyst.plans.QueryPlan
+import org.apache.spark.sql.catalyst.plans.physical.{HashPartitioning, Partitioning, UnknownPartitioning}
+import org.apache.spark.sql.execution.{DataSourceScanExec, WholeStageCodegenExec}
+import org.apache.spark.sql.execution.metric.SQLMetrics
+import org.apache.spark.sql.optimizer.CarbonFilters
+import org.apache.spark.sql.vectorized.ColumnarBatch
+
+import org.apache.carbondata.core.index.IndexFilter
+import org.apache.carbondata.core.indexstore.PartitionSpec
+import org.apache.carbondata.core.metadata.schema.BucketingInfo
+import org.apache.carbondata.core.readcommitter.ReadCommittedScope
+import org.apache.carbondata.core.scan.expression.Expression
+import org.apache.carbondata.core.scan.expression.logical.AndExpression
+import org.apache.carbondata.hadoop.CarbonProjection
+import org.apache.carbondata.spark.rdd.CarbonScanRDD
+
+/**
+ *  Physical plan node for scanning data. It is applied for both tables
+ *  USING carbondata and STORED AS carbondata.
+ */
+case class CarbonDataSourceScan(
+    @transient relation: CarbonDatasourceHadoopRelation,
+    output: Seq[Attribute],
+    partitionFilters: Seq[SparkExpression],
+    dataFilters: Seq[SparkExpression],
+    @transient readComittedScope: ReadCommittedScope,
+    @transient pushedDownProjection: CarbonProjection,
+    @transient pushedDownFilters: Seq[Expression],
+    directScanSupport: Boolean,
+    @transient extraRDD: Option[(RDD[InternalRow], Boolean)] = None,
+    tableIdentifier: Option[TableIdentifier] = None,
+    segmentIds: Option[String] = None)
+  extends DataSourceScanExec {
+
+  override lazy val supportsColumnar: Boolean = CarbonPlanHelper
+    .supportBatchedDataSource(sqlContext, output, extraRDD)
+
+  override protected def doExecuteColumnar(): RDD[ColumnarBatch] = {
+    val numOutputRows = longMetric("numOutputRows")
+    inputRDD.asInstanceOf[RDD[ColumnarBatch]].mapPartitionsInternal { batches =>
+      new Iterator[ColumnarBatch] {
+
+        override def hasNext: Boolean = {
+          val res = batches.hasNext
+          res
+        }
+
+        override def next(): ColumnarBatch = {
+          val batch = batches.next()
+          numOutputRows += batch.numRows()
+          batch
+        }
+      }
+    }
+  }
+
+  override lazy val metrics = Map(
+    "numOutputRows" -> SQLMetrics.createMetric(sparkContext, "number of output rows"))
+
+  lazy val needsUnsafeRowConversion: Boolean = { true }
+
+  override lazy val (outputPartitioning, outputOrdering): (Partitioning, Seq[SortOrder]) = {
+    val info: BucketingInfo = relation.carbonTable.getBucketingInfo
+    if (info != null) {
+      val cols = info.getListOfColumns.asScala
+      val numBuckets = info.getNumOfRanges
+      val bucketColumns = cols.flatMap { n =>
+        val attrRef = output.find(_.name.equalsIgnoreCase(n.getColumnName))
+        attrRef match {
+          case Some(attr: AttributeReference) =>
+            Some(AttributeReference(attr.name,
+              CarbonSparkDataSourceUtil.convertCarbonToSparkDataType(n.getDataType),
+              attr.nullable,
+              attr.metadata)(attr.exprId, attr.qualifier))
+          case _ => None
+        }
+      }
+      if (bucketColumns.size == cols.size) {
+        // use HashPartitioning will not shuffle
+        (HashPartitioning(bucketColumns, numBuckets), Nil)
+      } else {
+        (UnknownPartitioning(0), Nil)
+      }
+    } else {
+      (UnknownPartitioning(0), Nil)
+    }
+  }
+
+  override lazy val metadata: Map[String, String] = {
+    def seqToString(seq: Seq[Any]) = seq.mkString("[", ", ", "]")
+    val metadata =
+      Map(
+        "ReadSchema" -> seqToString(pushedDownProjection.getAllColumns),
+        "Batched" -> supportsColumnar.toString,
+        "DirectScan" -> (supportsColumnar && directScanSupport).toString,
+        "PushedFilters" -> seqToString(pushedDownFilters.map(_.getStatement)))
+    if (relation.carbonTable.isHivePartitionTable) {
+      metadata + ("PartitionFilters" -> seqToString(partitionFilters)) +
+        ("PartitionCount" -> selectedPartitions.size.toString)
+    } else {
+      metadata
+    }
+  }
+
+  @transient private lazy val indexFilter: IndexFilter = {
+    val filter = pushedDownFilters.reduceOption(new AndExpression(_, _))
+      .map(new IndexFilter(relation.carbonTable, _, true)).orNull
+    if (filter != null && pushedDownFilters.length == 1) {
+      // push down the limit if only one filter
+      filter.setLimit(relation.limit)
+    }
+    filter
+  }
+
+  @transient private lazy val selectedPartitions: Seq[PartitionSpec] = {
+    CarbonFilters
+      .getPartitions(partitionFilters, relation.sparkSession, relation.carbonTable)
+      .orNull
+  }
+
+  private lazy val inputRDD: RDD[InternalRow] = {
+    val carbonRdd = new CarbonScanRDD[InternalRow](
+      relation.sparkSession,
+      pushedDownProjection,
+      indexFilter,
+      relation.identifier,
+      relation.carbonTable.getTableInfo.serialize(),
+      relation.carbonTable.getTableInfo,
+      new CarbonInputMetrics,
+      selectedPartitions,
+      segmentIds = segmentIds)
+    carbonRdd.setVectorReaderSupport(supportsColumnar)
+    carbonRdd.setDirectScanSupport(supportsColumnar && directScanSupport)
+    extraRDD.map(_._1.union(carbonRdd)).getOrElse(carbonRdd)
+  }
+

Review comment:
       in this class also, common code we will move to common package, columnBatch we can keep for 3.1

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.spark.sql.parser
+
+import org.apache.spark.sql.{CarbonEnv, CarbonThreadUtil, SparkSession}
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkSqlParser
+import org.apache.spark.sql.internal.SQLConf
+import org.apache.spark.sql.util.CarbonException
+
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+/**
+ * parser order: carbon parser => spark parser
+ */
+class CarbonExtensionSqlParser(

Review comment:
       can combine 2.3 and 2.4

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala
##########
@@ -0,0 +1,199 @@
+/*
+ * 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.spark.sql.hive
+
+import java.util.concurrent.Callable
+
+import org.apache.hadoop.conf.Configuration
+import org.apache.hadoop.fs.Path
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition, ExternalCatalogWithListener, FunctionResourceLoader, GlobalTempViewManager}
+import org.apache.spark.sql.catalyst.expressions.Expression
+import org.apache.spark.sql.catalyst.parser.ParserInterface
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.connector.catalog.CatalogManager
+import org.apache.spark.sql.execution.strategy.{CarbonSourceStrategy, DDLStrategy, DMLStrategy, StreamingTableStrategy}
+import org.apache.spark.sql.hive.client.HiveClient
+import org.apache.spark.sql.internal.{SessionState, SQLConf}
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule}
+import org.apache.spark.sql.parser.CarbonSparkSqlParser
+
+import org.apache.carbondata.core.metadata.schema.table.CarbonTable
+
+/**
+ * This class will have carbon catalog and refresh the relation from cache if the carbontable in
+ * carbon catalog is not same as cached carbon relation's carbon table
+ *
+ * @param externalCatalog
+ * @param globalTempViewManager
+ * @param sparkSession
+ * @param functionResourceLoader
+ * @param functionRegistry
+ * @param conf
+ * @param hadoopConf
+ */
+class CarbonHiveSessionCatalog(

Review comment:
       can combine this class of 2.3 and 2.4 and move to common

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/DMLStrategy.scala
##########
@@ -240,6 +235,21 @@ object DMLStrategy extends SparkStrategy {
     condition.get.asInstanceOf[ScalaUDF].function.isInstanceOf[InPolygonJoinRangeListUDF]
   }
 
+  object CarbonExtractEquiJoinKeys {
+    def unapply(plan: LogicalPlan): Option[(JoinType, Seq[Expression], Seq[Expression],
+      Option[Expression], LogicalPlan, LogicalPlan)] = {
+      plan match {
+        case join: Join =>
+          ExtractEquiJoinKeys.unapply(join) match {
+              // ignoring hints as carbon is not using them right now

Review comment:
       add a TODO here to support join with Hints

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala
##########
@@ -0,0 +1,99 @@
+/*
+ * 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.spark.sql.hive
+
+import java.util
+
+import org.apache.spark.sql.CarbonToSparkAdapter
+
+import scala.collection.JavaConverters._
+import org.apache.spark.sql.catalyst.CarbonParserUtil
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{AddTableColumnsContext, CreateTableContext, HiveChangeColumnContext}
+import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, QualifiedColType}
+import org.apache.spark.sql.execution.SparkSqlAstBuilder
+import org.apache.spark.sql.execution.command.{AlterTableAddColumnsModel, AlterTableDataTypeChangeModel}
+import org.apache.spark.sql.execution.command.schema.{CarbonAlterTableAddColumnCommand, CarbonAlterTableColRenameDataTypeChangeCommand}
+import org.apache.spark.sql.execution.command.table.CarbonExplainCommand
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.types.{DecimalType, StructField}
+
+trait SqlAstBuilderHelper extends SparkSqlAstBuilder {
+
+

Review comment:
       remove extra line

##########
File path: mv/plan/src/main/spark2.3/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala
##########
@@ -47,4 +55,110 @@ object ExpressionHelper {
     reference.qualifier.head
   }
 
+  def getStatisticsObj(outputList: Seq[NamedExpression],
+      plan: LogicalPlan, stats: Statistics,
+      aliasMap: Option[AttributeMap[Attribute]] = None): Statistics = {
+    val output = outputList.map(_.toAttribute)
+    val mapSeq = plan.collect { case n: logical.LeafNode => n }.map {
+      table => AttributeMap(table.output.zip(output))
+    }
+    val rewrites = mapSeq.head
+    val attributes: AttributeMap[ColumnStat] = stats.attributeStats
+    var attributeStats = AttributeMap(attributes.iterator
+      .map { pair => (rewrites(pair._1), pair._2) }.toSeq)
+    if (aliasMap.isDefined) {
+      attributeStats = AttributeMap(
+        attributeStats.map(pair => (aliasMap.get(pair._1), pair._2)).toSeq)
+    }
+    Statistics(stats.sizeInBytes, stats.rowCount, attributeStats, stats.hints)
+  }
+
+  def getOptimizedPlan(s: SubqueryExpression): LogicalPlan = {
+    val Subquery(newPlan) = BirdcageOptimizer.execute(Subquery(s.plan))
+    newPlan
+  }
+
+  def normalizeExpressions(r: NamedExpression, attrs: AttributeSeq): NamedExpression = {
+    QueryPlan.normalizeExprId(r, attrs)
+  }
+
+  def attributeMap(rAliasMap: AttributeMap[Attribute]) : AttributeMap[Expression] = {
+    rAliasMap.asInstanceOf[AttributeMap[Expression]]
+  }
+
+  def seqOfRules : Seq[Rule[LogicalPlan]] = {
+    Seq(
+      // Operator push down
+      PushProjectionThroughUnion,
+      ReorderJoin,
+      EliminateOuterJoin,
+      PushPredicateThroughJoin,
+      PushDownPredicate,
+      ColumnPruning,
+      // Operator combine
+      CollapseRepartition,
+      CollapseProject,
+      CollapseWindow,
+      CombineFilters,
+      CombineLimits,
+      CombineUnions,
+      // Constant folding and strength reduction
+      NullPropagation,
+      FoldablePropagation,
+      ConstantFolding,
+      ReorderAssociativeOperator,
+      // No need to apply LikeSimplification rule while creating MV
+      // as modular plan asCompactSql will be set in schema
+      //        LikeSimplification,
+      BooleanSimplification,
+      SimplifyConditionals,
+      RemoveDispensableExpressions,
+      SimplifyBinaryComparison,
+      EliminateSorts,
+      SimplifyCasts,
+      SimplifyCaseConversionExpressions,
+      RewriteCorrelatedScalarSubquery,
+      EliminateSerialization,
+      RemoveRedundantAliases,
+      RemoveRedundantProject)
+  }
+}
+
+trait getVerboseString extends LeafNode {
+}
+
+trait groupByUnaryNode extends UnaryNode {
+}
+
+trait selectModularPlan extends ModularPlan {
+}
+
+trait unionModularPlan extends ModularPlan {
+}
+
+trait oneRowTableLeafNode extends LeafNode {
+}
+
+object MatchJoin {
+  def unapply(plan : LogicalPlan): Option[(LogicalPlan, LogicalPlan, JoinType, Option[Expression],
+    Option[Any])] = {
+    plan match {
+      case j@Join(left, right, joinType, condition) =>
+        val a = Some(left, right, joinType, condition, None)
+        a

Review comment:
       please refactor and move the common cod to common package fo Expression helper for all version

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/catalyst/CarbonDDLSqlParser.scala
##########
@@ -65,7 +65,10 @@ abstract class CarbonDDLSqlParser extends AbstractCarbonSparkSQLParser {
   protected val ESCAPECHAR = carbonKeyWord("ESCAPECHAR")
   protected val EXCLUDE = carbonKeyWord("EXCLUDE")
   protected val EXPLAIN = carbonKeyWord("EXPLAIN")
-  protected val EXTENDED = carbonKeyWord("EXTENDED")
+  protected val MODE = carbonKeyWord("EXTENDED") |
+                       carbonKeyWord("CODEGEN") |
+                       carbonKeyWord("COST") |
+                       carbonKeyWord("FORMATTED")

Review comment:
       please add test case with all MODE if not added

##########
File path: mv/plan/src/main/scala/org/apache/carbondata/mv/plans/util/Printers.scala
##########
@@ -358,6 +359,12 @@ trait Printers {
           ""
         }
         qualifierPrefix + quoteIdentifier(child.name) + " AS " + quoteIdentifier(a.name)
+      case a@Alias(child: AggregateExpression, _) =>
+        child.sql + " AS " + quoteIdentifier(a.name)

Review comment:
       use String concat using super, s"..."

##########
File path: processing/src/main/java/org/apache/carbondata/processing/loading/parser/impl/RowParserImpl.java
##########
@@ -109,7 +109,9 @@ public RowParserImpl(DataField[] output, CarbonDataLoadConfiguration configurati
       return new String[numberOfColumns];
     }
     // If number of columns are less in a row then create new array with same size of header.

Review comment:
       please update the comment according to new conditon

##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/SparkSqlAstBuilderWrapper.scala
##########
@@ -0,0 +1,30 @@
+/*
+ * 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.spark.sql.parser
+
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser._
+import org.apache.spark.sql.execution.SparkSqlAstBuilder
+import org.apache.spark.sql.internal.SQLConf
+
+/**
+ * use this wrapper to adapter multiple spark versions
+ */
+abstract class SparkSqlAstBuilderWrapper(conf: SQLConf) extends SparkSqlAstBuilder {

Review comment:
       can combine 2.3 and 2.4

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/joins/BroadCastSIFilterPushJoin.scala
##########
@@ -147,13 +152,15 @@ object BroadCastSIFilterPushJoin {
       inputCopy: Array[InternalRow],
       leftKeys: Seq[Expression],
       rightKeys: Seq[Expression],
-      buildSide: BuildSide,
+      buildSide: CarbonBuildSideType,
       isIndexTable: Boolean = false): Unit = {
 
+    val carbonBuildSide = CarbonBuildSide(buildSide)

Review comment:
       CarbonBuildSide can be put commonto 2.4 and 3.1 and refactor




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647125495



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/strategy/CarbonDataSourceScan.scala
##########
@@ -169,7 +128,7 @@ case class CarbonDataSourceScan(
   override protected def doCanonicalize(): CarbonDataSourceScan = {
     CarbonDataSourceScan(
       relation,
-      output.map(QueryPlan.normalizeExprId(_, output)),
+      a,

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647102930



##########
File path: integration/spark/src/main/spark2.3/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala
##########
@@ -21,21 +21,20 @@ import java.util.concurrent.Callable
 
 import org.apache.hadoop.conf.Configuration
 import org.apache.hadoop.fs.Path
+import org.apache.spark.sql.{CarbonEnv, SparkSession}
+import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier}
 import org.apache.spark.sql.catalyst.analysis.{Analyzer, FunctionRegistry}
 import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, CatalogTablePartition, FunctionResourceLoader, GlobalTempViewManager}
 import org.apache.spark.sql.catalyst.expressions.Expression
 import org.apache.spark.sql.catalyst.parser.ParserInterface
 import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
-import org.apache.spark.sql.catalyst.{QualifiedTableName, TableIdentifier}
 import org.apache.spark.sql.execution.strategy.{CarbonSourceStrategy, DDLStrategy, DMLStrategy, StreamingTableStrategy}
 import org.apache.spark.sql.hive.client.HiveClient
 import org.apache.spark.sql.internal.{SessionState, SQLConf}
 import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule}
 import org.apache.spark.sql.parser.CarbonSparkSqlParser
-import org.apache.spark.sql.{CarbonEnv, SparkSession}
 
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable
-import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
 

Review comment:
       Needed or else scalastyle fails




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647117148



##########
File path: integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -0,0 +1,294 @@
+/*
+ * 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.spark.sql
+
+import java.net.URI
+import java.sql.{Date, Timestamp}
+import java.time.ZoneId
+import javax.xml.bind.DatatypeConverter
+
+import scala.annotation.tailrec
+import scala.collection.mutable
+import scala.collection.mutable.ArrayBuffer
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd}
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, QueryPlanningTracker, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, AttributeSet, Expression, ExprId, NamedExpression, Predicate, ScalaUDF, SortOrder, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.codegen._
+import org.apache.spark.sql.catalyst.expressions.codegen.Block._
+import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide, Optimizer}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, PartitionFieldListContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, Statistics, SubqueryAlias}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.catalyst.util.{DateTimeUtils, TimestampFormatter}
+import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, ShuffledRowRDD, SimpleMode, SparkPlan, SQLExecution, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, RefreshTableCommand, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, FilePartition, PartitionedFile}
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.hive.HiveExternalCatalog
+import org.apache.spark.sql.optimizer.{CarbonIUDRule, CarbonUDFTransformRule, MVRewriteRule}
+import org.apache.spark.sql.secondaryindex.optimizer.CarbonSITransformationRule
+import org.apache.spark.sql.types.{AbstractDataType, CharType, DataType, Metadata, StringType, StructField, VarcharType}
+import org.apache.spark.sql.util.SparkSQLUtil

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647122347



##########
File path: mv/plan/src/main/common2.3and2.4/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala
##########
@@ -0,0 +1,137 @@
+/*
+ * 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.mv.plans.modular
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeReference, AttributeSeq, Expression, ExprId, NamedExpression, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate._
+import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, CollapseProject, CollapseRepartition, CollapseWindow, ColumnPruning, CombineFilters, CombineLimits, CombineUnions, ConstantFolding, EliminateOuterJoin, EliminateSerialization, EliminateSorts, FoldablePropagation, NullPropagation, PushDownPredicate, PushPredicateThroughJoin, PushProjectionThroughUnion, RemoveDispensableExpressions, RemoveRedundantAliases, RemoveRedundantProject, ReorderAssociativeOperator, ReorderJoin, RewriteCorrelatedScalarSubquery, SimplifyBinaryComparison, SimplifyCaseConversionExpressions, SimplifyCasts, SimplifyConditionals}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Join, LogicalPlan, Statistics, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+import org.apache.spark.sql.types.{DataType, Metadata}
+
+import org.apache.carbondata.mv.plans.util.BirdcageOptimizer
+
+
+object SparkVersionHelper {
+
+  def getStatisticsObj(outputList: Seq[NamedExpression],
+                       plan: LogicalPlan, stats: Statistics,
+                       aliasMap: Option[AttributeMap[Attribute]] = None): Statistics = {
+    val output = outputList.map(_.toAttribute)

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655892404



##########
File path: examples/spark/src/test/scala/org/apache/carbondata/examplesCI/RunExamples.scala
##########
@@ -104,7 +104,8 @@ class RunExamples extends QueryTest with BeforeAndAfterAll {
     TableLevelCompactionOptionExample.exampleBody(spark)
   }
 
-  test("LuceneIndexExample") {
+  // Below test case ignored due to the Deadlock in spark code

Review comment:
       Not reported as it is very specific to Carbondata and this issue is not reproduced in native spark code without carbondata.

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/IUDCommonUtil.scala
##########
@@ -44,7 +44,7 @@ object IUDCommonUtil {
       case unresolvedRelation: UnresolvedRelation =>
         val dbAndTb =
           sparkSession.sessionState.catalog.asInstanceOf[HiveSessionCatalog].getCurrentDatabase +
-          "." + unresolvedRelation.tableIdentifier.table
+          "." + CarbonToSparkAdapter.getTableIdentifier(unresolvedRelation).get.table

Review comment:
       Done

##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalysisRules.scala
##########
@@ -135,34 +137,34 @@ case class CarbonIUDAnalysisRule(sparkSession: SparkSession) extends Rule[Logica
       var transformed: Boolean = false
       // Create a dummy projection to include filter conditions
       var newPlan: LogicalPlan = null
-      if (table.tableIdentifier.database.isDefined) {
+      if (tableIdentifier.database.isDefined) {
         newPlan = parser.parsePlan("select * from  " +
-           table.tableIdentifier.database.getOrElse("") + "." +
-           table.tableIdentifier.table + " " + alias.getOrElse("") + " " + filter)
+           tableIdentifier.database.getOrElse("") + "." +

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647122743



##########
File path: mv/plan/src/main/spark2.3/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala
##########
@@ -46,5 +46,4 @@ object ExpressionHelper {
   def getTheLastQualifier(reference: AttributeReference): String = {
     reference.qualifier.head
   }
-
 }

Review comment:
       needed due to scalastyle

##########
File path: mv/plan/src/main/spark3.1/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala
##########
@@ -0,0 +1,166 @@
+/*
+ * 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.mv.plans.modular
+
+import scala.reflect.ClassTag
+
+import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeMap, AttributeSeq, Expression, ExprId, NamedExpression, SubqueryExpression}
+import org.apache.spark.sql.catalyst.expressions.aggregate.{AggregateExpression, AggregateFunction, AggregateMode}
+import org.apache.spark.sql.catalyst.optimizer.{BooleanSimplification, CollapseProject, CollapseRepartition, CollapseWindow, ColumnPruning, CombineFilters, CombineUnions, ConstantFolding, EliminateLimits, EliminateOuterJoin, EliminateSerialization, EliminateSorts, FoldablePropagation, NullPropagation, PushDownPredicates, PushPredicateThroughJoin, PushProjectionThroughUnion, RemoveDispensableExpressions, RemoveRedundantAliases, ReorderAssociativeOperator, ReorderJoin, RewriteCorrelatedScalarSubquery, SimplifyBinaryComparison, SimplifyCaseConversionExpressions, SimplifyCasts, SimplifyConditionals}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, Join, LogicalPlan, Statistics, Subquery}
+import org.apache.spark.sql.catalyst.rules.Rule
+
+import org.apache.carbondata.mv.plans.util.BirdcageOptimizer
+
+object SparkVersionHelper {
+  def getStatisticsObj(outputList: Seq[NamedExpression],
+      plan: LogicalPlan, stats: Statistics,
+      aliasMap: Option[AttributeMap[Attribute]] = None): Statistics = {
+    val output = outputList.map(_.toAttribute)
+    val mapSeq = plan.collect { case n: logical.LeafNode => n }.map {
+      table => AttributeMap(table.output.zip(output))
+    }
+    val rewrites = mapSeq.head
+    val attributes: AttributeMap[ColumnStat] = stats.attributeStats
+    var attributeStats = AttributeMap(attributes.iterator
+      .map { pair => (rewrites(pair._1), pair._2) }.toSeq)
+    if (aliasMap.isDefined) {
+      attributeStats = AttributeMap(
+        attributeStats.map(pair => (aliasMap.get(pair._1), pair._2)).toSeq)
+    }
+    Statistics(stats.sizeInBytes, stats.rowCount, attributeStats)
+  }
+
+  def getOptimizedPlan(s: SubqueryExpression): LogicalPlan = {
+    val Subquery(newPlan, _) = BirdcageOptimizer.execute(Subquery.fromExpression(s))
+    newPlan
+  }
+
+  def normalizeExpressions[T <: Expression](r: T, attrs: AttributeSeq): T = {
+    QueryPlan.normalizeExpressions(r, attrs)
+  }
+
+  def attributeMap(rAliasMap: AttributeMap[Attribute]) : AttributeMap[Alias] = {
+    rAliasMap.asInstanceOf[AttributeMap[Alias]]
+  }
+
+  def seqOfRules : Seq[Rule[LogicalPlan]] = {
+    Seq(
+      // Operator push down
+      PushProjectionThroughUnion,
+      ReorderJoin,
+      EliminateOuterJoin,
+      PushPredicateThroughJoin,
+      PushDownPredicates,
+      ColumnPruning,
+      // Operator combine
+      CollapseRepartition,
+      CollapseProject,
+      CollapseWindow,
+      CombineFilters,
+      EliminateLimits,
+      CombineUnions,
+      // Constant folding and strength reduction
+      NullPropagation,
+      FoldablePropagation,
+      ConstantFolding,
+      ReorderAssociativeOperator,
+      // No need to apply LikeSimplification rule while creating MV
+      // as modular plan asCompactSql will be set in schema
+      //        LikeSimplification,
+      BooleanSimplification,
+      SimplifyConditionals,
+      RemoveDispensableExpressions,
+      SimplifyBinaryComparison,
+      EliminateSorts,
+      SimplifyCasts,
+      SimplifyCaseConversionExpressions,
+      RewriteCorrelatedScalarSubquery,
+      EliminateSerialization,
+      RemoveRedundantAliases)
+  }
+
+}
+
+trait getVerboseString extends LeafNode {
+  def verboseString: String = toString
+}
+
+trait groupByUnaryNode extends UnaryNode {
+  override def verboseString(maxFields: Int): String = super.verboseString(maxFields)
+
+  override def mapProductIterator[B](f: Any => B)(implicit evidence$1: ClassTag[B]): Array[B] = {
+    super.mapProductIterator(f)
+  }
+
+  override def mapChildren(f: ModularPlan => ModularPlan) : ModularPlan = {
+    val groupBy = super.mapChildren(f)
+    if (this.rewritten && !groupBy.rewritten) {
+      groupBy.setRewritten()
+    }
+    groupBy
+  }
+}
+
+trait selectModularPlan extends ModularPlan {
+  override def verboseString(maxFields: Int): String = super.verboseString(maxFields)
+
+  override def mapProductIterator[B](f: Any => B)(implicit evidence$1: ClassTag[B]): Array[B] = {
+    super.mapProductIterator(f)
+  }
+
+  override def mapChildren(f: ModularPlan => ModularPlan) : ModularPlan = {
+    val select = super.mapChildren(f)
+    if (this.rewritten && !select.rewritten) {
+      select.setRewritten()
+    }
+    select
+  }
+}
+
+trait unionModularPlan extends ModularPlan {

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r645300638



##########
File path: integration/spark/src/main/scala/org/apache/carbondata/api/CarbonStore.scala
##########
@@ -360,9 +359,14 @@ object CarbonStore {
 
   private def validateTimeFormat(timestamp: String): Long = {
     try {
-      DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp)).get
+      CarbonToSparkAdapter.stringToTimestamp(timestamp) match {
+        case Some(value) => value
+        case _ =>
+          val errorMessage = "Error: Invalid load start time format: " + timestamp

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644562441



##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/CarbonToSparkAdapter.scala
##########
@@ -229,8 +270,388 @@ object CarbonToSparkAdapter {
       .unwrapped
       .asInstanceOf[HiveExternalCatalog]
   }
+
+  def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]): FilePartition = {
+    FilePartition(index, files.toArray)
+  }
+
+  def stringToTimestamp(timestamp: String): Option[Long] = {
+    DateTimeUtils.stringToTimestamp(UTF8String.fromString(timestamp))
+  }
+
+  def stringToTime(value: String): java.util.Date = {
+    DateTimeUtils.stringToTime(value)
+  }
+
+  def timeStampToString(timeStamp: Long): String = {
+    DateTimeUtils.timestampToString(timeStamp)
+  }
+
+  def getTableIdentifier(u: UnresolvedRelation): Some[TableIdentifier] = {
+    Some(u.tableIdentifier)
+  }
+
+  def dateToString(date: Int): String = {
+    DateTimeUtils.dateToString(date.toString.toInt)
+  }
+
+  def getProcessingTime: String => Trigger = {
+    Trigger.ProcessingTime
+  }
+
+  def addTaskCompletionListener[U](f: => U) {
+    TaskContext.get().addTaskCompletionListener { context =>
+      f
+    }
+  }
+
+  def createShuffledRowRDD(sparkContext: SparkContext, localTopK: RDD[InternalRow],
+      child: SparkPlan, serializer: Serializer): ShuffledRowRDD = {
+    new ShuffledRowRDD(
+      ShuffleExchangeExec.prepareShuffleDependency(
+        localTopK, child.output, SinglePartition, serializer))
+  }
+
+  def getInsertIntoCommand(table: LogicalPlan,
+      partition: Map[String, Option[String]],
+      query: LogicalPlan,
+      overwrite: Boolean,
+      ifPartitionNotExists: Boolean): InsertIntoTable = {
+    InsertIntoTable(
+      table,
+      partition,
+      query,
+      overwrite,
+      ifPartitionNotExists)
+  }
+
+  def getExplainCommandObj(logicalPlan: LogicalPlan = OneRowRelation(),
+      mode: Option[String]) : ExplainCommand = {
+    ExplainCommand(logicalPlan, mode.isDefined)
+  }
+
+  def invokeAnalyzerExecute(analyzer: Analyzer,
+      plan: LogicalPlan): LogicalPlan = {
+    analyzer.executeAndCheck(plan)
+  }
+
+  def normalizeExpressions(r: NamedExpression, attrs: AttributeSeq): NamedExpression = {
+    QueryPlan.normalizeExprId(r, attrs)
+  }
+
+  def getBuildRight: BuildSide = {
+    BuildRight
+  }
+
+  def getBuildLeft: BuildSide = {
+    BuildLeft
+  }
+
+  type CarbonBuildSideType = BuildSide
+  type InsertIntoStatementWrapper = InsertIntoTable
+
+  def withNewExecutionId[T](sparkSession: SparkSession, queryExecution: QueryExecution): T => T = {
+    SQLExecution.withNewExecutionId(sparkSession, queryExecution)(_)
+  }
+
+  def createJoinNode(child: LogicalPlan,
+      targetTable: LogicalPlan,
+      joinType: JoinType,
+      condition: Option[Expression]): Join = {
+    Join(child, targetTable, joinType, condition)
+  }
+
+  def getPartitionsFromInsert(x: InsertIntoStatementWrapper): Map[String, Option[String]] = {
+    x.partition
+  }
+
+  def getTableIdentifier(parts: TableIdentifier): TableIdentifier = {
+    parts
+  }
+
+  def getStatisticsObj(outputList: Seq[NamedExpression],
+      plan: LogicalPlan, stats: Statistics,
+      aliasMap: Option[AttributeMap[Attribute]] = None): Statistics = {
+    val output = outputList.map(_.toAttribute)
+    val mapSeq = plan.collect { case n: logical.LeafNode => n }.map {
+      table => AttributeMap(table.output.zip(output))
+    }
+    val rewrites = mapSeq.head
+    val attributes: AttributeMap[ColumnStat] = stats.attributeStats
+    var attributeStats = AttributeMap(attributes.iterator
+      .map { pair => (rewrites(pair._1), pair._2) }.toSeq)
+    if (aliasMap.isDefined) {
+      attributeStats = AttributeMap(
+        attributeStats.map(pair => (aliasMap.get(pair._1), pair._2)).toSeq)
+    }
+    Statistics(stats.sizeInBytes, stats.rowCount, attributeStats, stats.hints)
+  }
+
+  def createRefreshTableCommand(tableIdentifier: TableIdentifier): RefreshTable = {
+    RefreshTable(tableIdentifier)
+  }
+
+  type RefreshTables = RefreshTable

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655893002



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSpark2SqlParser.scala
##########
@@ -835,10 +841,21 @@ class CarbonSpark2SqlParser extends CarbonDDLSqlParser {
         field.dataType = Some("double")
       }
       field.rawSchema = scannerInput
-      if (col.getComment().isDefined) {
+      if (comment.isDefined) {
         field.columnComment = plainComment
       }
       field
+  }
+
+  def getScannerInput(dataType: DataType,
+                      columnComment: String,
+                      columnName: String,
+                      isExternal: Boolean): String = {

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



[GitHub] [carbondata] vikramahuja1001 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-889018791


   Hi @Jeromestein , you can get in touch with us via the mailing list as well you can also slack to get in touch with the community. [Mailing List](http://apache-carbondata-dev-mailing-list-archive.1130556.n5.nabble.com/) and [Slack Link](carbondataworkspace.slack.com
   ) 
   You can check the Subtasks of the main [Spark-3.1 Jira](https://issues.apache.org/jira/browse/CARBONDATA-4190) and start working on them.


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

To unsubscribe, e-mail: dev-unsubscribe@carbondata.apache.org

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-865995078


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


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

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-852859571


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r647119204



##########
File path: integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/SparkVersionAdapter.scala
##########
@@ -0,0 +1,440 @@
+/*
+ * 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.spark.sql
+
+import scala.collection.mutable
+
+import org.antlr.v4.runtime.tree.TerminalNode
+import org.apache.spark.{SparkContext, TaskContext}
+import org.apache.spark.rdd.RDD
+import org.apache.spark.serializer.Serializer
+import org.apache.spark.sql.catalyst.{CarbonParserUtil, InternalRow, TableIdentifier}
+import org.apache.spark.sql.catalyst.analysis.{Analyzer, UnresolvedRelation}
+import org.apache.spark.sql.catalyst.encoders.RowEncoder
+import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeMap, AttributeSeq, Expression, NamedExpression, SortOrder}
+import org.apache.spark.sql.catalyst.expressions.codegen.{ExprCode, GeneratePredicate}
+import org.apache.spark.sql.catalyst.parser.ParserUtils.operationNotAllowed
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{BucketSpecContext, ColTypeListContext, CreateTableHeaderContext, LocationSpecContext, QueryContext, SkewSpecContext, TablePropertyListContext}
+import org.apache.spark.sql.catalyst.plans.{logical, JoinType, QueryPlan}
+import org.apache.spark.sql.catalyst.plans.logical.{ColumnStat, InsertIntoTable, Join, LogicalPlan, OneRowRelation, Statistics}
+import org.apache.spark.sql.catalyst.plans.physical.SinglePartition
+import org.apache.spark.sql.catalyst.util.DateTimeUtils
+import org.apache.spark.sql.execution.{QueryExecution, ShuffledRowRDD, SparkPlan, SQLExecution, UnaryExecNode}
+import org.apache.spark.sql.execution.command.{ExplainCommand, Field, PartitionerField, TableModel, TableNewProcessor}
+import org.apache.spark.sql.execution.command.table.{CarbonCreateTableAsSelectCommand, CarbonCreateTableCommand}
+import org.apache.spark.sql.execution.datasources.{DataSourceStrategy, RefreshTable}
+import org.apache.spark.sql.execution.exchange.ShuffleExchangeExec
+import org.apache.spark.sql.execution.joins.{BuildLeft, BuildRight, BuildSide}
+import org.apache.spark.sql.execution.strategy.CarbonDataSourceScan
+import org.apache.spark.sql.internal.{SessionState, SharedState}
+import org.apache.spark.sql.parser.CarbonSpark2SqlParser
+import org.apache.spark.sql.parser.CarbonSparkSqlParserUtil.{checkIfDuplicateColumnExists, convertDbNameToLowerCase, validateStreamingProperty}
+import org.apache.spark.sql.sources.Filter
+import org.apache.spark.sql.types.{DataType, StructField}
+import org.apache.spark.unsafe.types.UTF8String
+
+import org.apache.carbondata.common.exceptions.DeprecatedFeatureException
+import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
+import org.apache.carbondata.core.constants.CarbonCommonConstants
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier
+import org.apache.carbondata.core.metadata.datatype.DataTypes
+import org.apache.carbondata.core.metadata.schema.SchemaReader
+import org.apache.carbondata.core.util.CarbonProperties
+import org.apache.carbondata.core.util.path.CarbonTablePath
+import org.apache.carbondata.spark.CarbonOption
+import org.apache.carbondata.spark.util.CarbonScalaUtil
+
+object SparkVersionAdapter {

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



[GitHub] [carbondata] kunal642 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
kunal642 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r646323700



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonExtensionSpark2SqlParser.scala
##########
@@ -56,4 +58,13 @@ class CarbonExtensionSpark2SqlParser extends CarbonSpark2SqlParser {
         CarbonSparkSqlParserUtil.loadDataNew(
           databaseNameOp, tableName, Option(optionsList), partitions, filePath, isOverwrite)
     }
+
+  /**
+   * REFRESH MATERIALIZED VIEW mv_name
+   */
+  private lazy val refreshTable: Parser[LogicalPlan] =

Review comment:
       why is this needed?




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644544057



##########
File path: integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala
##########
@@ -501,40 +321,6 @@ object CarbonSparkSqlParserUtil {
     }
   }
 
-  /**
-   * Validates the partition columns and return's A tuple of partition columns and partitioner
-   * fields.
-   *
-   * @param partitionColumns        An instance of ColTypeListContext having parser rules for
-   *                                column.
-   * @param colNames                <Seq[String]> Sequence of Table column names.
-   * @param tableProperties         <Map[String, String]> Table property map.
-   * @param partitionByStructFields Seq[StructField] Sequence of partition fields.
-   * @return <Seq[PartitionerField]> A Seq of partitioner fields.
-   */
-  def validatePartitionFields(

Review comment:
       added to common package




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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r645302372



##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/CarbonBoundReference.scala
##########
@@ -42,4 +42,4 @@ case class CarbonBoundReference(colExp: ColumnExpression, dataType: DataType, nu
   override def qualifier: Seq[String] = null
 
   override def newInstance(): NamedExpression = throw new UnsupportedOperationException
-}
\ No newline at end of file
+}

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644563505



##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala
##########
@@ -0,0 +1,52 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.SparkSession
+import org.apache.spark.sql.catalyst.parser.ParserUtils.string
+import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{AddTableColumnsContext, CreateHiveTableContext}
+import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan
+import org.apache.spark.sql.execution.SparkSqlAstBuilder
+import org.apache.spark.sql.internal.SQLConf

Review comment:
       done

##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala
##########
@@ -0,0 +1,90 @@
+/*
+ * 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.spark.sql.hive
+
+import org.apache.spark.sql.CarbonToSparkAdapter

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-852977056


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r644563841



##########
File path: integration/spark/src/main/spark2.4/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala
##########
@@ -0,0 +1,81 @@
+/*
+ * 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.spark.sql.parser
+

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



[GitHub] [carbondata] CarbonDataQA2 commented on pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
CarbonDataQA2 commented on pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#issuecomment-848232531


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


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

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



[GitHub] [carbondata] vikramahuja1001 commented on a change in pull request #4141: [CARBONDATA-4190] Integrate Carbondata with Spark 3.1.1 version

Posted by GitBox <gi...@apache.org>.
vikramahuja1001 commented on a change in pull request #4141:
URL: https://github.com/apache/carbondata/pull/4141#discussion_r655295712



##########
File path: examples/spark/src/main/scala/org/apache/carbondata/examples/LuceneIndexExample.scala
##########
@@ -60,40 +60,6 @@ object LuceneIndexExample {
          | AS 'lucene'
       """.stripMargin)
 
-    // 1. Compare the performance:
-
-    def time(code: => Unit): Double = {
-      val start = System.currentTimeMillis()
-      code
-      // return time in second
-      (System.currentTimeMillis() - start).toDouble / 1000
-    }
-
-    val timeWithoutLuceneIndex = time {
-
-      spark.sql(
-        s"""
-           | SELECT count(*)
-           | FROM personTable where id like '% test1 %'
-      """.stripMargin).show()
-
-    }
-
-    val timeWithLuceneIndex = time {
-
-      spark.sql(

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