From 739a5b1f30f2d9ce1fa7a82a836ac6c5ba99a197 Mon Sep 17 00:00:00 2001 From: QiangCai Date: Wed, 10 Apr 2024 21:47:24 +0800 Subject: [PATCH] [WIP][ISSUE-4346] Support Spark 3.5 --- .../core/reader/CarbonIndexFileReader.java | 2 +- .../carbondata/core/reader/ThriftReader.java | 14 +- .../carbondata/core/util/CarbonUtil.java | 6 +- .../carbondata/core/writer/ThriftWriter.java | 12 +- examples/spark/pom.xml | 22 - format/pom.xml | 2 +- index/examples/pom.xml | 22 - index/secondary-index/pom.xml | 21 - integration/flink/pom.xml | 77 +-- integration/spark/pom.xml | 126 ----- .../sql/CarbonDataSourceScanHelper.scala | 89 ---- .../spark/sql/SparkVersionAdapter.scala | 470 ------------------ .../sql/execution/CarbonCodegenSupport.scala | 23 - .../spark/sql/hive/CarbonAnalyzer.scala | 50 -- .../spark/sql/hive/CarbonSqlAstBuilder.scala | 52 -- .../spark/sql/hive/SqlAstBuilderHelper.scala | 89 ---- .../command/CarbonResetCommand.scala | 46 -- .../sql/parser/CarbonExtensionSqlParser.scala | 82 --- .../sql/parser/CarbonSparkSqlParser.scala | 152 ------ .../parser/SparkSqlAstBuilderWrapper.scala | 31 -- .../spark/sql/CarbonBoundReference.scala | 45 -- .../carbondata/spark/load/CsvRDDHelper.scala | 11 +- .../carbondata/spark/util/CommonUtil.scala | 8 +- .../apache/carbondata/streamer/Source.scala | 1 - .../spark/sql/CarbonBoundReference.scala | 27 +- .../sql/CarbonDataSourceScanHelper.scala | 2 +- .../spark/sql/CarbonToSparkAdapter.scala | 3 +- .../apache/spark/sql/SparkSqlAdapter.scala | 6 + .../spark/sql/SparkVersionAdapter.scala | 32 +- .../sql/avro/AvroFileFormatFactory.scala | 0 .../sql/execution/CarbonCodegenSupport.scala | 0 .../CarbonTakeOrderedAndProjectExec.scala | 3 + .../CreateDataSourceTableCommand.scala | 8 +- .../command/management/CommonLoadUtils.scala | 17 +- .../merge/CarbonMergeDataSetCommand.scala | 1 - .../command/mutation/merge/MergeHandler.scala | 1 - .../spark/sql/hive/CarbonAnalyzer.scala | 0 .../sql/hive/CarbonSessionStateBuilder.scala | 23 +- .../spark/sql/hive/CarbonSqlAstBuilder.scala | 4 +- .../spark/sql/hive/SqlAstBuilderHelper.scala | 11 +- .../command/CarbonHiveCommands.scala | 10 + .../command/CarbonResetCommand.scala | 5 + .../sql/parser/CarbonExtensionSqlParser.scala | 0 .../sql/parser/CarbonSparkSqlParser.scala | 17 +- .../sql/parser/CarbonSparkSqlParserUtil.scala | 4 +- .../hive/CarbonInternalMetastore.scala | 4 +- .../databricks/spark/avro/AvroWriter.scala | 51 -- .../spark/sql/CarbonBoundReference.scala | 45 -- .../spark/sql/CarbonToSparkAdapter.scala | 230 --------- .../apache/spark/sql/SparkSqlAdapter.scala | 50 -- .../sql/avro/AvroFileFormatFactory.scala | 49 -- .../CreateDataSourceTableCommand.scala | 30 -- .../sql/hive/CarbonSessionStateBuilder.scala | 189 ------- .../spark/sql/CarbonToSparkAdapter.scala | 272 ---------- .../sql/hive/CarbonSessionStateBuilder.scala | 198 -------- .../apache/spark/sql/SparkSqlAdapter.scala | 49 -- .../parser/SparkSqlAstBuilderWrapper.scala | 30 -- .../src/test/resources/datawithoutheader.csv | 10 - .../view/rewrite/TestAllOperationsOnMV.scala | 2 + mv/plan/pom.xml | 125 ----- .../mv/plans/modular/SparkVersionHelper.scala | 137 ----- .../mv/expressions/modular/subquery.scala | 12 +- .../mv/plans/modular/AggregatePushDown.scala | 4 +- .../mv/plans/modular/ExpressionHelper.scala | 16 +- .../mv/plans/modular/ModularPlan.scala | 5 + .../mv/plans/modular/SparkVersionHelper.scala | 2 +- .../mv/plans/modular/ExpressionHelper.scala | 50 -- .../mv/plans/modular/ExpressionHelper.scala | 58 --- pom.xml | 237 +-------- streaming/pom.xml | 118 ----- .../streaming/parser/RowStreamParserImp.scala | 9 +- .../carbondata/util/SparkStreamingUtil.scala | 0 .../SparkStreamingUtil.scala | 35 -- 73 files changed, 196 insertions(+), 3448 deletions(-) delete mode 100644 integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/CarbonDataSourceScanHelper.scala delete mode 100644 integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/SparkVersionAdapter.scala delete mode 100644 integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/execution/CarbonCodegenSupport.scala delete mode 100644 integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/CarbonAnalyzer.scala delete mode 100644 integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala delete mode 100644 integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala delete mode 100644 integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala delete mode 100644 integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala delete mode 100644 integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala delete mode 100644 integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/parser/SparkSqlAstBuilderWrapper.scala delete mode 100644 integration/spark/src/main/common2.4and3.1/org/apache/spark/sql/CarbonBoundReference.scala rename integration/spark/src/main/{spark3.1 => scala}/org/apache/spark/sql/CarbonDataSourceScanHelper.scala (98%) rename integration/spark/src/main/{spark3.1 => scala}/org/apache/spark/sql/CarbonToSparkAdapter.scala (98%) rename integration/spark/src/main/{spark2.4 => scala}/org/apache/spark/sql/SparkSqlAdapter.scala (88%) rename integration/spark/src/main/{spark3.1 => scala}/org/apache/spark/sql/SparkVersionAdapter.scala (93%) rename integration/spark/src/main/{common2.4and3.1 => scala}/org/apache/spark/sql/avro/AvroFileFormatFactory.scala (100%) rename integration/spark/src/main/{spark3.1 => scala}/org/apache/spark/sql/execution/CarbonCodegenSupport.scala (100%) rename integration/spark/src/main/{common2.4and3.1 => scala}/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala (92%) rename integration/spark/src/main/{spark3.1 => scala}/org/apache/spark/sql/hive/CarbonAnalyzer.scala (100%) rename integration/spark/src/main/{spark3.1 => scala}/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala (91%) rename integration/spark/src/main/{spark3.1 => scala}/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala (93%) rename integration/spark/src/main/{spark3.1 => scala}/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala (92%) rename integration/spark/src/main/{spark3.1 => scala}/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala (92%) rename integration/spark/src/main/{spark3.1 => scala}/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala (100%) rename integration/spark/src/main/{spark3.1 => scala}/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala (94%) delete mode 100644 integration/spark/src/main/spark2.3/com/databricks/spark/avro/AvroWriter.scala delete mode 100644 integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonBoundReference.scala delete mode 100644 integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapter.scala delete mode 100644 integration/spark/src/main/spark2.3/org/apache/spark/sql/SparkSqlAdapter.scala delete mode 100644 integration/spark/src/main/spark2.3/org/apache/spark/sql/avro/AvroFileFormatFactory.scala delete mode 100644 integration/spark/src/main/spark2.3/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala delete mode 100644 integration/spark/src/main/spark2.3/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala delete mode 100644 integration/spark/src/main/spark2.4/org/apache/spark/sql/CarbonToSparkAdapter.scala delete mode 100644 integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala delete mode 100644 integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkSqlAdapter.scala delete mode 100644 integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/SparkSqlAstBuilderWrapper.scala delete mode 100644 integration/spark/src/test/resources/datawithoutheader.csv delete mode 100644 mv/plan/src/main/common2.3and2.4/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala rename mv/plan/src/main/{spark2.4 => scala}/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala (85%) rename mv/plan/src/main/{spark3.1 => scala}/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala (90%) delete mode 100644 mv/plan/src/main/spark2.3/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala delete mode 100644 mv/plan/src/main/spark3.1/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala rename streaming/src/main/{spark3.1 => scala}/org/apache/carbondata/util/SparkStreamingUtil.scala (100%) delete mode 100644 streaming/src/main/spark2.x/org.apache.carbondata.util/SparkStreamingUtil.scala diff --git a/core/src/main/java/org/apache/carbondata/core/reader/CarbonIndexFileReader.java b/core/src/main/java/org/apache/carbondata/core/reader/CarbonIndexFileReader.java index b99b13bfe1e..9639e3b1597 100644 --- a/core/src/main/java/org/apache/carbondata/core/reader/CarbonIndexFileReader.java +++ b/core/src/main/java/org/apache/carbondata/core/reader/CarbonIndexFileReader.java @@ -87,7 +87,7 @@ public void openThriftReader(String filePath) throws IOException { * * @param fileData */ - public void openThriftReader(byte[] fileData) { + public void openThriftReader(byte[] fileData) throws IOException { thriftReader = new ThriftReader(fileData); } diff --git a/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java b/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java index d0873237a68..f44f6944644 100644 --- a/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java +++ b/core/src/main/java/org/apache/carbondata/core/reader/ThriftReader.java @@ -87,9 +87,13 @@ public ThriftReader(String fileName, Configuration configuration) { /** * Constructor. */ - public ThriftReader(byte[] fileData) { + public ThriftReader(byte[] fileData) throws IOException { dataInputStream = new DataInputStream(new ByteArrayInputStream(fileData)); - binaryIn = new TCompactProtocol(new TIOStreamTransport(dataInputStream)); + try { + binaryIn = new TCompactProtocol(new TIOStreamTransport(dataInputStream)); + } catch (TException e) { + throw new IOException(e); + } } /** @@ -98,7 +102,11 @@ public ThriftReader(byte[] fileData) { public void open() throws IOException { Configuration conf = configuration != null ? configuration : FileFactory.getConfiguration(); dataInputStream = FileFactory.getDataInputStream(fileName, conf); - binaryIn = new TCompactProtocol(new TIOStreamTransport(dataInputStream)); + try { + binaryIn = new TCompactProtocol(new TIOStreamTransport(dataInputStream)); + } catch (TException e) { + throw new IOException(e); + } } /** diff --git a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java index 19040cf84ea..bf01c81795f 100644 --- a/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java +++ b/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java @@ -1413,8 +1413,8 @@ public static String convertToString(List values) { public static byte[] getByteArray(TBase t) { ByteArrayOutputStream stream = new ByteArrayOutputStream(); byte[] thriftByteArray = null; - TProtocol binaryOut = new TCompactProtocol(new TIOStreamTransport(stream)); try { + TProtocol binaryOut = new TCompactProtocol(new TIOStreamTransport(stream)); t.write(binaryOut); stream.flush(); thriftByteArray = stream.toByteArray(); @@ -1439,9 +1439,9 @@ public static DataChunk3 readDataChunk3(ByteBuffer dataChunkBuffer, int offset, public static DataChunk3 readDataChunk3(InputStream stream) throws IOException { TBaseCreator creator = DataChunk3::new; - TProtocol binaryIn = new TCompactProtocol(new TIOStreamTransport(stream)); TBase t = creator.create(); try { + TProtocol binaryIn = new TCompactProtocol(new TIOStreamTransport(stream)); t.read(binaryIn); } catch (TException e) { throw new IOException(e); @@ -1461,9 +1461,9 @@ public static DataChunk3 readDataChunk3(InputStream stream) throws IOException { private static TBase read(byte[] data, TBaseCreator creator, int offset, int length) throws IOException { ByteArrayInputStream stream = new ByteArrayInputStream(data, offset, length); - TProtocol binaryIn = new TCompactProtocol(new TIOStreamTransport(stream)); TBase t = creator.create(); try { + TProtocol binaryIn = new TCompactProtocol(new TIOStreamTransport(stream)); t.read(binaryIn); } catch (TException e) { throw new IOException(e); diff --git a/core/src/main/java/org/apache/carbondata/core/writer/ThriftWriter.java b/core/src/main/java/org/apache/carbondata/core/writer/ThriftWriter.java index 3314aaf9eb9..389ef69e023 100644 --- a/core/src/main/java/org/apache/carbondata/core/writer/ThriftWriter.java +++ b/core/src/main/java/org/apache/carbondata/core/writer/ThriftWriter.java @@ -80,7 +80,11 @@ public ThriftWriter(String fileName, boolean append) { */ public void open() throws IOException { dataOutputStream = FileFactory.getDataOutputStream(fileName, bufferSize, append); - binaryOut = new TCompactProtocol(new TIOStreamTransport(dataOutputStream)); + try { + binaryOut = new TCompactProtocol(new TIOStreamTransport(dataOutputStream)); + } catch (TException e) { + throw new IOException(e); + } } /** @@ -92,7 +96,11 @@ public void open() throws IOException { public void open(FileWriteOperation fileWriteOperation) throws IOException { atomicFileOperationsWriter = AtomicFileOperationFactory.getAtomicFileOperations(fileName); dataOutputStream = atomicFileOperationsWriter.openForWrite(fileWriteOperation); - binaryOut = new TCompactProtocol(new TIOStreamTransport(dataOutputStream)); + try { + binaryOut = new TCompactProtocol(new TIOStreamTransport(dataOutputStream)); + } catch (TException e) { + throw new IOException(e); + } } /** diff --git a/examples/spark/pom.xml b/examples/spark/pom.xml index 9a891487c24..58759d29b71 100644 --- a/examples/spark/pom.xml +++ b/examples/spark/pom.xml @@ -198,27 +198,5 @@ true - - spark-2.3 - - 2.3 - - - - spark-2.4 - - true - - - 2.4 - - - - spark-3.1 - - 3.1 - 2.10.0 - - diff --git a/format/pom.xml b/format/pom.xml index 2ba968977c0..291c0be3368 100644 --- a/format/pom.xml +++ b/format/pom.xml @@ -37,7 +37,7 @@ org.apache.thrift libthrift - 0.9.3 + 0.19.0 diff --git a/index/examples/pom.xml b/index/examples/pom.xml index 80e778b9db4..9857c6e7b92 100644 --- a/index/examples/pom.xml +++ b/index/examples/pom.xml @@ -77,26 +77,4 @@ - - - - spark-2.3 - - 2.3 - - - - spark-2.4 - - 2.4 - - - - spark-3.1 - - 3.1 - - - - \ No newline at end of file diff --git a/index/secondary-index/pom.xml b/index/secondary-index/pom.xml index 15cb7a7b849..8caaa1a4b66 100644 --- a/index/secondary-index/pom.xml +++ b/index/secondary-index/pom.xml @@ -156,27 +156,6 @@ true - - spark-2.3 - - 2.3 - - - - spark-2.4 - - true - - - 2.4 - - - - spark-3.1 - - 3.1 - - diff --git a/integration/flink/pom.xml b/integration/flink/pom.xml index ba82c3302ce..ca33eb333f9 100644 --- a/integration/flink/pom.xml +++ b/integration/flink/pom.xml @@ -22,12 +22,12 @@ org.apache.carbondata - carbondata-flink-proxy + carbondata-format ${project.version} org.apache.carbondata - carbondata-format + carbondata-flink-proxy ${project.version} @@ -215,72 +215,21 @@ 4.1.17.Final test - - - - - spark-2.3 - - 2.3 - - - - org.apache.carbondata - carbondata-spark_${spark.binary.version} - ${project.version} - test - - - org.apache.hive - hive-exec - - - - - - - spark-2.4 - - true - - - 2.4 - - - - org.apache.carbondata - carbondata-spark_${spark.binary.version} - ${project.version} - test - - - org.apache.hive - hive-exec - - - - - - - spark-3.1 - - 3.1 - - - + org.apache.carbondata carbondata-spark_${spark.binary.version} ${project.version} test - - - com.thoughtworks.paranamer - paranamer - - - - - + + + com.thoughtworks.paranamer + paranamer + + + + + + sdvtest diff --git a/integration/spark/pom.xml b/integration/spark/pom.xml index 0c3b9d9d74e..600f515e972 100644 --- a/integration/spark/pom.xml +++ b/integration/spark/pom.xml @@ -606,131 +606,5 @@ true - - spark-2.3 - - 2.3 - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - src/main/spark3.1 - src/main/spark2.4 - src/main/common2.4and3.1 - - - - - org.codehaus.mojo - build-helper-maven-plugin - 3.0.0 - - - add-source - generate-sources - - add-source - - - - src/main/spark2.3 - src/main/common2.3and2.4 - - - - - - - - - - spark-2.4 - - true - - - 2.4 - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - src/main/spark3.1 - src/main/spark2.3 - - - - - org.codehaus.mojo - build-helper-maven-plugin - 3.0.0 - - - add-source - generate-sources - - add-source - - - - src/main/spark2.4 - src/main/common2.3and2.4 - src/main/common2.4and3.1 - - - - - - - - - - spark-3.1 - - 3.1 - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - src/main/spark2.4 - src/main/spark2.3 - src/main/common2.3and2.4 - - - - - org.codehaus.mojo - build-helper-maven-plugin - 3.0.0 - - - add-source - generate-sources - - add-source - - - - src/main/spark3.1 - src/main/common2.4and3.1 - - - - - - - - diff --git a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/CarbonDataSourceScanHelper.scala b/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/CarbonDataSourceScanHelper.scala deleted file mode 100644 index 22183dec0c4..00000000000 --- a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/CarbonDataSourceScanHelper.scala +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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.catalog.CatalogTablePartition -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)], - selectedCatalogPartitions: Seq[CatalogTablePartition], - partitionFilterWithDpp: Seq[SparkExpression], - segmentIds: Option[String]) - extends DataSourceScanExec with ColumnarBatchScan { - - override lazy val supportsBatch: Boolean = { - CarbonPlanHelper.supportBatchedDataSource(sqlContext, output, extraRDD) - } - - lazy val supportsBatchOrColumnar: Boolean = supportsBatch - - val outputAttibutesAfterNormalizingExpressionIds: Seq[Attribute] = output - .map(QueryPlan.normalizeExprId(_, output)) - - @transient 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 lazy val selectedPartitions: Seq[PartitionSpec] = { - CarbonFilters - .getPartitions(partitionFilters, relation.sparkSession, relation.carbonTable) - .orNull - } - - 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(supportsBatch) - carbonRdd.setDirectScanSupport(supportsBatch && directScanSupport) - extraRDD.map(_._1.union(carbonRdd)).getOrElse(carbonRdd) - } -} diff --git a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/SparkVersionAdapter.scala b/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/SparkVersionAdapter.scala deleted file mode 100644 index e5cb5e0b13b..00000000000 --- a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/SparkVersionAdapter.scala +++ /dev/null @@ -1,470 +0,0 @@ -/* - * 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, AttributeSeq, Expression, InterpretedPredicate, NamedExpression, SortOrder} -import org.apache.spark.sql.catalyst.expressions.codegen.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.{InsertIntoTable, Join, LogicalPlan, OneRowRelation} -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.{CreateTable, 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 - -trait SparkVersionAdapter { - - def getPredicate(inputSchema: Seq[Attribute], - condition: Option[Expression]): InternalRow => Boolean = { - GeneratePredicate.generate(condition.get, inputSchema).eval(_) - } - - 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 rebaseTime(timestamp: Long, carbonWrittenVersion: String = null): Long = { - // From spark 3.1, spark will store gregorian micros value for timestamp, hence - // rebase is required. For 2.x versions, no need rebase - timestamp - } - - def stringToTime(value: String): java.util.Date = { - DateTimeUtils.stringToTime(value) - } - - 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 createRefreshTableCommand(tableIdentifier: TableIdentifier): RefreshTable = { - RefreshTable(tableIdentifier) - } - - type RefreshTables = RefreshTable - - - /** - * 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 Sequence of Table column names. - * @param tableProperties Table property map. - * @param partitionByStructFields Seq[StructField] Sequence of partition fields. - * @return A Seq of partitioner fields. - */ - def validatePartitionFields( - partitionColumns: ColTypeListContext, - 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: ColTypeListContext) - } - } - 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, ColTypeListContext, 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 of create table or ctas table - * - */ - def createCarbonTable(createTableTuple: (CreateTableHeaderContext, SkewSpecContext, - BucketSpecContext, ColTypeListContext, 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[StructField], - isExternal: Boolean = false): Seq[Field] = { - schema.map { col => - parser.getFields(col.getComment, col.name, col.dataType, isExternal) - } - } - - def supportsBatchOrColumnar(scan: CarbonDataSourceScan): Boolean = { - scan.supportsBatch - } - - def createDataset(sparkSession: SparkSession, qe: QueryExecution) : Dataset[Row] = { - new Dataset[Row](sparkSession, qe, RowEncoder(qe.analyzed.schema)) - } - - def createSharedState(sparkContext: SparkContext) : SharedState = { - new SharedState(sparkContext) - } - - def translateFilter(dataFilters: Seq[Expression]) : Seq[Filter] = { - dataFilters.flatMap(DataSourceStrategy.translateFilter) - } - - def getCarbonOptimizer(session : SparkSession, sessionState: SessionState) : CarbonOptimizer = { - new CarbonOptimizer(session, sessionState.catalog, sessionState.optimizer) - } - - def isCharType(dataType: DataType): Boolean = { - false - } - - def isVarCharType(dataType: DataType): Boolean = { - false - } - - def getTypeName(s: DataType): String = { - s.typeName - } - - def evaluateWithPredicate(exp: Expression, schema: Seq[Attribute], row: InternalRow): Any = { - InterpretedPredicate.create(exp, schema).expression.eval(row) - } - - def getUpdatedPlan(plan: LogicalPlan, sqlText: String): LogicalPlan = { - plan match { - case create@CreateTable(tableDesc, mode, query) => - if ( tableDesc.storage.locationUri.isDefined && - !sqlText.toUpperCase.startsWith("CREATE EXTERNAL TABLE ")) { - // add a property to differentiate if create table statement has external keyword or not - val newProperties = tableDesc.properties. +("hasexternalkeyword" -> "false") - val updatedTableDesc = tableDesc.copy(properties = newProperties) - CreateTable(updatedTableDesc, mode, query) - } else if (tableDesc.storage.properties.contains("latestversion")) { - val newProperties = tableDesc.storage - .properties.filterNot(_._1.equalsIgnoreCase("latestversion")) - val updatedStorage = tableDesc.storage.copy(properties = newProperties) - CreateTable(tableDesc.copy(storage = updatedStorage), mode, query) - } else { - create - } - case others => others - } - } -} - -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: String = { - val orderByString = sortOrder.mkString("[", ",", "]") - val outputString = output.mkString("[", ",", "]") - - s"CarbonTakeOrderedAndProjectExec(limit=$limit, orderBy=$orderByString, " + - s"skipMapOrder=$skipMapOrder, readFromHead=$readFromHead, output=$outputString)" - } -} diff --git a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/execution/CarbonCodegenSupport.scala b/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/execution/CarbonCodegenSupport.scala deleted file mode 100644 index 4fceb8b183b..00000000000 --- a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/execution/CarbonCodegenSupport.scala +++ /dev/null @@ -1,23 +0,0 @@ -/* - * 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 { - -} diff --git a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/CarbonAnalyzer.scala b/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/CarbonAnalyzer.scala deleted file mode 100644 index bfa2e98f705..00000000000 --- a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/CarbonAnalyzer.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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, - conf: SQLConf, - sparkSession: SparkSession, - analyzer: Analyzer) extends Analyzer(catalog, conf) { - - val mvPlan = try { - CarbonReflectionUtils.createObject( - "org.apache.spark.sql.optimizer.MVRewriteRule", - sparkSession)._1.asInstanceOf[Rule[LogicalPlan]] - } catch { - case e: Exception => - null - } - - override def execute(plan: LogicalPlan): LogicalPlan = { - val logicalPlan = analyzer.execute(plan) - if (mvPlan != null) { - mvPlan.apply(logicalPlan) - } else { - logicalPlan - } - } -} diff --git a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala b/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala deleted file mode 100644 index 36ec2c806e1..00000000000 --- a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala +++ /dev/null @@ -1,52 +0,0 @@ -/* - * 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 -import org.apache.spark.sql.parser.{CarbonHelperSqlAstBuilder, CarbonSpark2SqlParser, CarbonSparkSqlParserUtil} - -class CarbonSqlAstBuilder(conf: SQLConf, parser: CarbonSpark2SqlParser, sparkSession: SparkSession) - extends SparkSqlAstBuilder(conf) with SqlAstBuilderHelper { - - val helper = new CarbonHelperSqlAstBuilder(conf, parser, sparkSession) - - override def visitCreateHiveTable(ctx: CreateHiveTableContext): LogicalPlan = { - val fileStorage = CarbonSparkSqlParserUtil.getFileStorage(ctx.createFileFormat(0)) - - if (fileStorage.equalsIgnoreCase("'carbondata'") || - fileStorage.equalsIgnoreCase("carbondata") || - fileStorage.equalsIgnoreCase("'carbonfile'") || - fileStorage.equalsIgnoreCase("'org.apache.carbondata.format'")) { - val createTableTuple = (ctx.createTableHeader, ctx.skewSpec(0), - ctx.bucketSpec(0), ctx.partitionColumns, ctx.columns, ctx.tablePropertyList(0), - ctx.locationSpec(0), Option(ctx.STRING(0)).map(string), ctx.AS, ctx.query, fileStorage) - helper.createCarbonTable(createTableTuple) - } else { - super.visitCreateHiveTable(ctx) - } - } - - override def visitAddTableColumns(ctx: AddTableColumnsContext): LogicalPlan = { - visitAddTableColumns(parser, ctx) - } -} diff --git a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala b/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala deleted file mode 100644 index a8cbbafb0f5..00000000000 --- a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala +++ /dev/null @@ -1,89 +0,0 @@ -/* - * 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 -import org.apache.spark.sql.catalyst.CarbonParserUtil -import org.apache.spark.sql.catalyst.parser.SqlBaseParser -import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{AddTableColumnsContext, ChangeColumnContext, CreateTableContext} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -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 - -trait SqlAstBuilderHelper extends SparkSqlAstBuilder { - - override def visitChangeColumn(ctx: ChangeColumnContext): LogicalPlan = { - - val newColumn = visitColType(ctx.colType) - val isColumnRename = !ctx.identifier.getText.equalsIgnoreCase(newColumn.name) - - val (typeString, values): (String, Option[List[(Int, Int)]]) = newColumn.dataType match { - case d: DecimalType => ("decimal", Some(List((d.precision, d.scale)))) - case _ => (newColumn.dataType.typeName.toLowerCase, None) - } - - val alterTableColRenameAndDataTypeChangeModel = - AlterTableDataTypeChangeModel( - CarbonParserUtil.parseDataType(newColumn.name, typeString, values), - CarbonParserUtil.convertDbNameToLowerCase(Option(ctx.tableIdentifier().db).map(_.getText)), - ctx.tableIdentifier().table.getText.toLowerCase, - ctx.identifier.getText.toLowerCase, - newColumn.name.toLowerCase, - isColumnRename) - - CarbonAlterTableColRenameDataTypeChangeCommand(alterTableColRenameAndDataTypeChangeModel) - } - - - def visitAddTableColumns(parser: CarbonSpark2SqlParser, - ctx: AddTableColumnsContext): LogicalPlan = { - val cols = Option(ctx.columns).toSeq.flatMap(visitColTypeList) - val fields = CarbonToSparkAdapter.getField(parser, cols) - val tblProperties = scala.collection.mutable.Map.empty[String, String] - val tableModel = CarbonParserUtil.prepareTableModel(false, - CarbonParserUtil.convertDbNameToLowerCase(Option(ctx.tableIdentifier().db).map(_.getText)), - ctx.tableIdentifier.table.getText.toLowerCase, - fields, - Seq.empty, - tblProperties, - None, - true) - - val alterTableAddColumnsModel = AlterTableAddColumnsModel( - Option(ctx.tableIdentifier().db).map(_.getText), - ctx.tableIdentifier.table.getText, - tblProperties.toMap, - tableModel.dimCols, - tableModel.msrCols, - tableModel.highCardinalityDims.getOrElse(Seq.empty)) - - CarbonAlterTableAddColumnCommand(alterTableAddColumnsModel) - } - - override def visitCreateTable(ctx: CreateTableContext): LogicalPlan = { - super.visitCreateTable(ctx) - } - - override def visitExplain(ctx: SqlBaseParser.ExplainContext): LogicalPlan = { - CarbonExplainCommand(super.visitExplain(ctx)) - } -} diff --git a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala b/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala deleted file mode 100644 index eab772d9fb6..00000000000 --- a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala +++ /dev/null @@ -1,46 +0,0 @@ -/* - * 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, Row, SparkSession} -import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan -import org.apache.spark.sql.execution.command.{ResetCommand, RunnableCommand} - -case class CarbonResetCommand() - extends RunnableCommand { - override val output = ResetCommand.output - - override def run(sparkSession: SparkSession): Seq[Row] = { - CarbonEnv.getInstance(sparkSession).carbonSessionInfo.getSessionParams.clear() - ResetCommand.run(sparkSession) - } -} - -/** - * This method matches the reset command based on the spark version - */ -object MatchResetCommand { - def unapply(plan: LogicalPlan): Option[LogicalPlan] = { - plan match { - case r@ResetCommand => - Some(plan) - case _ => - None - } - } -} diff --git a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala b/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala deleted file mode 100644 index 18d0f1bdf95..00000000000 --- a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala +++ /dev/null @@ -1,82 +0,0 @@ -/* - * 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, CarbonToSparkAdapter, 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( - conf: SQLConf, - sparkSession: SparkSession, - initialParser: ParserInterface -) extends SparkSqlParser(conf) { - - val parser = new CarbonExtensionSpark2SqlParser - val antlrParser = new CarbonAntlrParser(this) - - override def parsePlan(sqlText: String): LogicalPlan = { - parser.synchronized { - CarbonEnv.getInstance(sparkSession) - } - CarbonThreadUtil.updateSessionInfoToCurrentThread(sparkSession) - try { - parser.parse(sqlText) - } catch { - case ce: MalformedCarbonCommandException => - throw ce - case ct: Throwable => - try { - antlrParser.parse(sqlText) - } catch { - case ce: MalformedCarbonCommandException => - throw ce - case at: Throwable => - try { - val parsedPlan = CarbonToSparkAdapter.getUpdatedPlan(initialParser.parsePlan(sqlText), - sqlText) - CarbonScalaUtil.cleanParserThreadLocals - parsedPlan - } catch { - case mce: MalformedCarbonCommandException => - throw mce - case st: Throwable => - st.printStackTrace(System.err) - CarbonScalaUtil.cleanParserThreadLocals - CarbonException.analysisException( - s"""== Spark Parser: ${initialParser.getClass.getName} == - |${st.getMessage} - |== Carbon Parser: ${ parser.getClass.getName } == - |${ct.getMessage} - |== Antlr Parser: ${antlrParser.getClass.getName} == - |${at.getMessage} - """.stripMargin.trim) - } - } - } - } -} diff --git a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala b/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala deleted file mode 100644 index 6fbd4b62d3d..00000000000 --- a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala +++ /dev/null @@ -1,152 +0,0 @@ -/* - * 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.parser.CarbonSparkSqlParserUtil.convertPropertiesToLowercase -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(conf) { - - val parser = new CarbonSpark2SqlParser - - override val astBuilder = CarbonReflectionUtils.getAstBuilder(conf, parser, sparkSession) - - private val substitutor = new VariableSubstitution(conf) - - 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, ColTypeListContext, ColTypeListContext, TablePropertyListContext, - LocationSpecContext, Option[String], TerminalNode, QueryContext, String)): LogicalPlan = { - // val parser = new CarbonSpark2SqlParser - - val (tableHeader, skewSpecContext, - bucketSpecContext, - partitionColumns, - columns, - tablePropertyList, - locationSpecContext, - tableComment, - ctas, - query, - provider) = createTableTuple - - val (tableIdent, temp, ifNotExists, external) = visitCreateTableHeader(tableHeader) - val tableIdentifier = CarbonToSparkAdapter.getTableIdentifier(tableIdent) - val cols: Seq[StructField] = Option(columns).toSeq.flatMap(visitColTypeList) - val colNames: Seq[String] = CarbonSparkSqlParserUtil - .validateCreateTableReqAndGetColumns(tableHeader, - skewSpecContext, - bucketSpecContext, - columns, - cols, - tableIdentifier, - temp) - val tablePath: Option[String] = if (locationSpecContext != null) { - Some(visitLocationSpec(locationSpecContext)) - } else { - None - } - - val properties: Map[String, String] = getPropertyKeyValues(tablePropertyList) - val tableProperties = convertPropertiesToLowercase(properties) - // validate partition clause - val partitionByStructFields = Option(partitionColumns).toSeq.flatMap(visitColTypeList) - val partitionFields = CarbonToSparkAdapter. - validatePartitionFields(partitionColumns, colNames, tableProperties, - partitionByStructFields) - - // validate for create table as select - val selectQuery = Option(query).map(plan) - val extraTableTuple = (cols, external, tableIdentifier, ifNotExists, colNames, tablePath, - tableProperties, properties, partitionByStructFields, partitionFields, - parser, sparkSession, selectQuery) - CarbonToSparkAdapter.createCarbonTable(createTableTuple, extraTableTuple) - } -} - -trait CarbonAstTrait { - def getFileStorage (createFileFormat : CreateFileFormatContext): String -} - - diff --git a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/parser/SparkSqlAstBuilderWrapper.scala b/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/parser/SparkSqlAstBuilderWrapper.scala deleted file mode 100644 index 5c4de438ada..00000000000 --- a/integration/spark/src/main/common2.3and2.4/org/apache/spark/sql/parser/SparkSqlAstBuilderWrapper.scala +++ /dev/null @@ -1,31 +0,0 @@ -/* - * 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(conf) { - - def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] -} diff --git a/integration/spark/src/main/common2.4and3.1/org/apache/spark/sql/CarbonBoundReference.scala b/integration/spark/src/main/common2.4and3.1/org/apache/spark/sql/CarbonBoundReference.scala deleted file mode 100644 index 8efdb25ecf1..00000000000 --- a/integration/spark/src/main/common2.4and3.1/org/apache/spark/sql/CarbonBoundReference.scala +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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) - extends LeafExpression with NamedExpression with CodegenFallback { - - type EvaluatedType = Any - - override def toString: String = s"input[" + colExp.getColIndex + "]" - - override def eval(input: InternalRow): Any = input.get(colExp.getColIndex, dataType) - - override def name: String = colExp.getColumnName - - override def toAttribute: Attribute = throw new UnsupportedOperationException - - override def exprId: ExprId = throw new UnsupportedOperationException - - override def qualifier: Seq[String] = null - - override def newInstance(): NamedExpression = throw new UnsupportedOperationException -} diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/load/CsvRDDHelper.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/load/CsvRDDHelper.scala index 4867e5f7ded..778cb13516b 100644 --- a/integration/spark/src/main/scala/org/apache/carbondata/spark/load/CsvRDDHelper.scala +++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/load/CsvRDDHelper.scala @@ -25,6 +25,7 @@ import org.apache.hadoop.fs.Path 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.paths.SparkPath import org.apache.spark.rdd.RDD import org.apache.spark.sql.{CarbonToSparkAdapter, SparkSession} import org.apache.spark.sql.catalyst.InternalRow @@ -70,7 +71,7 @@ object CsvRDDHelper { totalLength = totalLength + fileSplit.getLength PartitionedFile( InternalRow.empty, - fileSplit.getPath.toString, + SparkPath.fromPath(fileSplit.getPath), fileSplit.getStart, fileSplit.getLength, fileSplit.getLocations) @@ -111,7 +112,7 @@ object CsvRDDHelper { // 2. read function val readFunction = getReadFunction(hadoopConf) - new FileScanRDD(spark, readFunction, partitions) + new FileScanRDD(spark, readFunction, partitions, null) } /** @@ -132,7 +133,7 @@ object CsvRDDHelper { val tableBlock = distributable.asInstanceOf[TableBlockInfo] PartitionedFile( InternalRow.empty, - tableBlock.getFilePath, + SparkPath.fromPathString(tableBlock.getFilePath), tableBlock.getBlockOffset, tableBlock.getBlockLength, tableBlock.getLocations) @@ -146,7 +147,7 @@ object CsvRDDHelper { // 2. read function val readFunction = getReadFunction(hadoopConf) - new FileScanRDD(spark, readFunction, partitions) + new FileScanRDD(spark, readFunction, partitions, null) } private def getReadFunction(configuration: Configuration): (PartitionedFile => @@ -161,7 +162,7 @@ object CsvRDDHelper { val hadoopAttemptContext = new TaskAttemptContextImpl(FileFactory.getConfiguration, attemptId) val inputSplit = - new FileSplit(new Path(file.filePath), file.start, file.length, file.locations) + new FileSplit(file.filePath.toPath, file.start, file.length, file.locations) var finished = false val inputFormat = new CSVInputFormat() val reader = inputFormat.createRecordReader(inputSplit, hadoopAttemptContext) diff --git a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala index 5b3914b9b58..c8ad3d82a45 100644 --- a/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala +++ b/integration/spark/src/main/scala/org/apache/carbondata/spark/util/CommonUtil.scala @@ -20,7 +20,6 @@ package org.apache.carbondata.spark.util import java.io.{ByteArrayOutputStream, DataOutputStream, File} import java.math.BigDecimal import java.text.SimpleDateFormat -import java.util import java.util.UUID import java.util.regex.{Matcher, Pattern} @@ -29,27 +28,24 @@ import scala.collection.mutable import scala.collection.mutable.Map import scala.math.BigDecimal.RoundingMode +import java.util import org.apache.hadoop.conf.Configuration import org.apache.hadoop.mapreduce.lib.input.FileInputFormat import org.apache.spark.{SparkContext, SparkEnv} import org.apache.spark.sql.catalyst.{InternalRow, TableIdentifier} import org.apache.spark.sql.catalyst.expressions.{UnsafeArrayData, UnsafeMapData, UnsafeRow} import org.apache.spark.sql.execution.command.{ColumnProperty, Field, PartitionerField} -import org.apache.spark.sql.types.{ArrayType, DataType, DateType, DecimalType, MapType, StringType, StructField, StructType, TimestampType} +import org.apache.spark.sql.types._ import org.apache.spark.util.FileUtils import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException import org.apache.carbondata.common.logging.LogServiceFactory import org.apache.carbondata.core.constants.CarbonCommonConstants -import org.apache.carbondata.core.datastore.impl.FileFactory import org.apache.carbondata.core.keygenerator.directdictionary.timestamp.DateDirectDictionaryGenerator import org.apache.carbondata.core.memory.{UnsafeMemoryManager, UnsafeSortMemoryManager} -import org.apache.carbondata.core.metadata.CarbonMetadata import org.apache.carbondata.core.metadata.datatype.DataTypes import org.apache.carbondata.core.metadata.schema.table.CarbonTable -import org.apache.carbondata.core.statusmanager.SegmentStatusManager import org.apache.carbondata.core.util.{CarbonProperties, CarbonUtil, DataTypeUtil, ThreadLocalTaskInfo} -import org.apache.carbondata.core.util.path.CarbonTablePath import org.apache.carbondata.processing.datatypes.{ArrayDataType, GenericDataType, StructDataType} import org.apache.carbondata.processing.loading.CarbonDataLoadConfiguration import org.apache.carbondata.processing.loading.complexobjects.{ArrayObject, StructObject} diff --git a/integration/spark/src/main/scala/org/apache/carbondata/streamer/Source.scala b/integration/spark/src/main/scala/org/apache/carbondata/streamer/Source.scala index 6103f57f7be..b8b3671c219 100644 --- a/integration/spark/src/main/scala/org/apache/carbondata/streamer/Source.scala +++ b/integration/spark/src/main/scala/org/apache/carbondata/streamer/Source.scala @@ -24,7 +24,6 @@ import org.apache.avro.Schema import org.apache.avro.Schema.Type import org.apache.avro.generic.GenericRecord import org.apache.spark.sql.{CarbonEnv, Row, SparkSession} -import org.apache.spark.sql.avro.{AvroDeserializer, SchemaConverters} import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.GenericRowWithSchema diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonBoundReference.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonBoundReference.scala index 589e49c71f0..2acc1fc55e0 100644 --- a/integration/spark/src/main/scala/org/apache/spark/sql/CarbonBoundReference.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonBoundReference.scala @@ -17,7 +17,32 @@ package org.apache.spark.sql -import org.apache.spark.sql.catalyst.expressions.Expression +import org.apache.spark.sql.catalyst.InternalRow +import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, 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) + extends LeafExpression with NamedExpression with CodegenFallback { + + type EvaluatedType = Any + + override def toString: String = s"input[" + colExp.getColIndex + "]" + + override def eval(input: InternalRow): Any = input.get(colExp.getColIndex, dataType) + + override def name: String = colExp.getColumnName + + override def toAttribute: Attribute = throw new UnsupportedOperationException + + override def exprId: ExprId = throw new UnsupportedOperationException + + override def qualifier: Seq[String] = null + + override def newInstance(): NamedExpression = throw new UnsupportedOperationException +} object ExtractReferences { def apply(expr: Expression): Array[String] = { diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonDataSourceScanHelper.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDataSourceScanHelper.scala similarity index 98% rename from integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonDataSourceScanHelper.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/CarbonDataSourceScanHelper.scala index 20a8f5bf30a..5d73161fcab 100644 --- a/integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonDataSourceScanHelper.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonDataSourceScanHelper.scala @@ -21,7 +21,7 @@ import org.apache.spark.CarbonInputMetrics import org.apache.spark.rdd.RDD import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.{CatalogTablePartition, ExternalCatalogUtils} -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression => SparkExpression, PlanExpression} +import org.apache.spark.sql.catalyst.expressions.{Attribute, PlanExpression, Expression => SparkExpression} import org.apache.spark.sql.catalyst.expressions.codegen.CodegenContext import org.apache.spark.sql.execution.{DataSourceScanExec, WholeStageCodegenExec} import org.apache.spark.sql.execution.metric.SQLMetrics diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonToSparkAdapter.scala similarity index 98% rename from integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/CarbonToSparkAdapter.scala index 9ab78880e2b..690a723a7cf 100644 --- a/integration/spark/src/main/spark3.1/org/apache/spark/sql/CarbonToSparkAdapter.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/CarbonToSparkAdapter.scala @@ -26,7 +26,8 @@ import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, ExternalCatalogWithListener} -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSet, DynamicPruningSubquery, Expression, ExprId, NamedExpression, Predicate, ScalaUDF, SubqueryExpression} +import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, + AttributeSet, DynamicPruningSubquery, Expression, ExprId, NamedExpression, Predicate, ScalaUDF} import org.apache.spark.sql.catalyst.expressions.BindReferences.bindReference import org.apache.spark.sql.catalyst.expressions.codegen._ import org.apache.spark.sql.catalyst.expressions.codegen.Block._ diff --git a/integration/spark/src/main/spark2.4/org/apache/spark/sql/SparkSqlAdapter.scala b/integration/spark/src/main/scala/org/apache/spark/sql/SparkSqlAdapter.scala similarity index 88% rename from integration/spark/src/main/spark2.4/org/apache/spark/sql/SparkSqlAdapter.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/SparkSqlAdapter.scala index e1c804e275d..d3ce3e2d33c 100644 --- a/integration/spark/src/main/spark2.4/org/apache/spark/sql/SparkSqlAdapter.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/SparkSqlAdapter.scala @@ -19,6 +19,7 @@ package org.apache.spark.sql import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.FileSourceScanExec import org.apache.spark.sql.execution.datasources.HadoopFsRelation import org.apache.spark.sql.types.StructType @@ -42,7 +43,12 @@ object SparkSqlAdapter { outputSchema, partitionFilters, None, + None, dataFilters, tableIdentifier) } + + def ofRows(sparkSession: SparkSession, logicalPlan: LogicalPlan): DataFrame = { + Dataset.ofRows(sparkSession, logicalPlan) + } } diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkVersionAdapter.scala b/integration/spark/src/main/scala/org/apache/spark/sql/SparkVersionAdapter.scala similarity index 93% rename from integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkVersionAdapter.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/SparkVersionAdapter.scala index 805af13a683..be95bc659ff 100644 --- a/integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkVersionAdapter.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/SparkVersionAdapter.scala @@ -34,9 +34,12 @@ import org.apache.spark.sql.catalyst.encoders.RowEncoder import org.apache.spark.sql.catalyst.expressions.{Attribute, AttributeSeq, Expression, Predicate, SortOrder} import org.apache.spark.sql.catalyst.optimizer.{BuildLeft, BuildRight, BuildSide} 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.parser.SqlBaseParser._ import org.apache.spark.sql.catalyst.plans.{JoinType, QueryPlan} -import org.apache.spark.sql.catalyst.plans.logical.{CreateTableStatement, InsertIntoStatement, Join, JoinHint, LogicalPlan, OneRowRelation, QualifiedColType} +import org.apache.spark.sql.catalyst.plans.logical.{ + CreateTable, InsertIntoStatement, Join, + JoinHint, LogicalPlan, OneRowRelation, QualifiedColType, TableSpec +} import org.apache.spark.sql.catalyst.plans.physical.SinglePartition import org.apache.spark.sql.catalyst.util.{DateTimeUtils, RebaseDateTime, TimestampFormatter} import org.apache.spark.sql.execution.{ExplainMode, QueryExecution, ShuffledRowRDD, SimpleMode, SparkPlan, SQLExecution, UnaryExecNode} @@ -61,7 +64,7 @@ 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.CarbonProperties import org.apache.carbondata.core.util.path.CarbonTablePath import org.apache.carbondata.spark.CarbonOption import org.apache.carbondata.spark.util.CarbonScalaUtil @@ -255,7 +258,7 @@ trait SparkVersionAdapter { * */ def createCarbonTable(createTableTuple: (CreateTableHeaderContext, SkewSpecContext, - BucketSpecContext, PartitionFieldListContext, ColTypeListContext, TablePropertyListContext, + BucketSpecContext, PartitionFieldListContext, ColTypeListContext, PropertyListContext, 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], @@ -429,7 +432,7 @@ trait SparkVersionAdapter { } def createDataset(sparkSession: SparkSession, qe: QueryExecution) : Dataset[Row] = { - new Dataset[Row](qe, RowEncoder(qe.analyzed.schema)) + new Dataset[Row](qe, RowEncoder.encoderFor(qe.analyzed.schema)) } def createSharedState(sparkContext: SparkContext) : SharedState = { @@ -474,21 +477,16 @@ trait SparkVersionAdapter { def getUpdatedPlan(plan: LogicalPlan, sqlText: String): LogicalPlan = { plan match { - case create@CreateTableStatement(_, _, _, _, properties, _, _, - location, _, _, _, _) => - if ( location.isDefined && + case create@CreateTable(_, _, _, table@TableSpec(properties, _, _, location, _, _, _), _) => + if (location.isDefined && !sqlText.toUpperCase.startsWith("CREATE EXTERNAL TABLE ")) { // add a property to differentiate if create table statement has external keyword or not - val newProperties = properties. +("hasexternalkeyword" -> "false") - CreateTableStatement(create.tableName, create.tableSchema, create.partitioning, - create.bucketSpec, newProperties, create.provider, create.options, - location, create.comment, create.serde, create.external, create.ifNotExists) - } else if (create.options.contains("latestversion")) { + val newProperties = properties + ("hasexternalkeyword" -> "false") + create.copy(tableSpec = table.copy(properties = newProperties)) + } else if (table.options.contains("latestversion")) { // remove latestversion property in options if present - val newOptions = create.options.filterNot(_._1.equalsIgnoreCase("latestversion")) - CreateTableStatement(create.tableName, create.tableSchema, create.partitioning, - create.bucketSpec, properties, create.provider, newOptions, - location, create.comment, create.serde, create.external, create.ifNotExists) + val newOptions = table.options.filterNot(_._1.equalsIgnoreCase("latestversion")) + create.copy(tableSpec = table.copy(options = newOptions)) } else { create } diff --git a/integration/spark/src/main/common2.4and3.1/org/apache/spark/sql/avro/AvroFileFormatFactory.scala b/integration/spark/src/main/scala/org/apache/spark/sql/avro/AvroFileFormatFactory.scala similarity index 100% rename from integration/spark/src/main/common2.4and3.1/org/apache/spark/sql/avro/AvroFileFormatFactory.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/avro/AvroFileFormatFactory.scala diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/execution/CarbonCodegenSupport.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/CarbonCodegenSupport.scala similarity index 100% rename from integration/spark/src/main/spark3.1/org/apache/spark/sql/execution/CarbonCodegenSupport.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/execution/CarbonCodegenSupport.scala diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/CarbonTakeOrderedAndProjectExec.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/CarbonTakeOrderedAndProjectExec.scala index 8102a0c6aae..d743a7613f0 100644 --- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/CarbonTakeOrderedAndProjectExec.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/CarbonTakeOrderedAndProjectExec.scala @@ -113,4 +113,7 @@ case class CarbonTakeOrderedAndProjectExec( } } + override protected def withNewChildInternal(newChild: SparkPlan): SparkPlan = { + withNewChildren(Seq(newChild)) + } } diff --git a/integration/spark/src/main/common2.4and3.1/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala similarity index 92% rename from integration/spark/src/main/common2.4and3.1/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala index c1a7d7f2e64..b2a5be48572 100644 --- a/integration/spark/src/main/common2.4and3.1/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala @@ -19,7 +19,8 @@ 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.catalyst.catalog.{CatalogTable, CatalogTableType} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.command.RunnableCommand import org.apache.spark.sql.util.CreateTableCommonUtil.getCatalogTable @@ -48,6 +49,11 @@ case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boo sessionState.catalog.createTable(newTable, ignoreIfExists = false, validateLocation = false) Seq.empty[Row] } + + override protected def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]) + : LogicalPlan = { + this + } } object CreateDataSourceTableCommand { diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CommonLoadUtils.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CommonLoadUtils.scala index 9a75895d1e6..0c43d799ea8 100644 --- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CommonLoadUtils.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/CommonLoadUtils.scala @@ -460,7 +460,8 @@ object CommonLoadUtils { isNoRearrangeFlow: Boolean, table: CarbonTable, partition: Map[String, Option[String]]): (LogicalPlan, Int, Option[RDD[InternalRow]]) = { - val catalogAttributes = catalogTable.schema.toAttributes + val catalogAttributes = catalogTable.schema.map( + x => AttributeReference(x.name, x.dataType, x.nullable, x.metadata)()) // Converts the data as per the loading steps before give it to writer or sorter var attributes = curAttributes.map(a => { catalogAttributes.find(_.name.equalsIgnoreCase(a.name)).get @@ -746,12 +747,14 @@ object CommonLoadUtils { if (options.contains(DataLoadProcessorConstants.NO_REARRANGE_OF_ROWS)) { CarbonReflectionUtils.getLogicalRelation(hdfsRelation, - metastoreSchema.toAttributes, + metastoreSchema.map( + x => AttributeReference(x.name, x.dataType, x.nullable, x.metadata)()), Some(catalogTable), false) } else { CarbonReflectionUtils.getLogicalRelation(hdfsRelation, - hdfsRelation.schema.toAttributes, + hdfsRelation.schema.map( + x => AttributeReference(x.name, x.dataType, x.nullable, x.metadata)()), Some(catalogTable), false) } @@ -872,7 +875,8 @@ object CommonLoadUtils { CarbonThreadUtil.threadSet("partition.operationcontext", loadParams.operationContext) val attributes = if (loadParams.scanResultRDD.isDefined) { // take the already re-arranged attributes - catalogTable.schema.toAttributes + catalogTable.schema.map( + x => AttributeReference(x.name, x.dataType, x.nullable, x.metadata)()) } else { // input data from csv files. Convert to logical plan val allCols = new ArrayBuffer[String]() @@ -881,7 +885,8 @@ object CommonLoadUtils { allCols ++= table.getVisibleMeasures.asScala.map(_.getColName) StructType( allCols.filterNot(_.equals(CarbonCommonConstants.DEFAULT_INVISIBLE_DUMMY_MEASURE)).map( - StructField(_, StringType))).toAttributes + StructField(_, StringType))).map( + x => AttributeReference(x.name, x.dataType, x.nullable, x.metadata)()) } var partitionsLen = 0 val sortScope = CarbonDataProcessorUtil.getSortScope(loadParams.carbonLoadModel.getSortScope) @@ -1093,7 +1098,7 @@ object CommonLoadUtils { overwrite = false, ifPartitionNotExists = false) SparkUtil.setNullExecutionId(loadParams.sparkSession) - Dataset.ofRows(loadParams.sparkSession, convertedPlan).collect() + SparkSqlAdapter.ofRows(loadParams.sparkSession, convertedPlan).collect() } catch { case ex: Throwable => val (executorMessage, errorMessage) = CarbonScalaUtil.retrieveAndLogErrorMsg(ex, LOGGER) diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/CarbonMergeDataSetCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/CarbonMergeDataSetCommand.scala index 53f5c26612f..06ac2257593 100644 --- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/CarbonMergeDataSetCommand.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/CarbonMergeDataSetCommand.scala @@ -30,7 +30,6 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.spark.rdd.RDD import org.apache.spark.sql.{AnalysisException, CarbonEnv, CarbonToSparkAdapter, Column, DataFrame, Dataset, Row, SparkSession} -import org.apache.spark.sql.avro.AvroFileFormatFactory import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.analysis.UnresolvedAttribute import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression, GenericInternalRow} diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/MergeHandler.scala b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/MergeHandler.scala index e75f11b265b..68dbe3d635b 100644 --- a/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/MergeHandler.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/execution/command/mutation/merge/MergeHandler.scala @@ -27,7 +27,6 @@ import org.apache.hadoop.mapreduce.lib.output.FileOutputFormat import org.apache.hadoop.mapreduce.task.TaskAttemptContextImpl import org.apache.spark.rdd.RDD import org.apache.spark.sql.{DataFrame, Row, SparkSession} -import org.apache.spark.sql.avro.AvroFileFormatFactory import org.apache.spark.sql.catalyst.InternalRow import org.apache.spark.sql.catalyst.expressions.GenericInternalRow import org.apache.spark.sql.execution.command.{ExecutionErrors, UpdateTableModel} diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/CarbonAnalyzer.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalyzer.scala similarity index 100% rename from integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/CarbonAnalyzer.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonAnalyzer.scala diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala similarity index 91% rename from integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala index 5ecc38183c4..d5c3a9859fb 100644 --- a/integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala @@ -23,12 +23,15 @@ 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.analysis.{Analyzer, FunctionRegistry, TableFunctionRegistry} +import org.apache.spark.sql.catalyst.catalog.{ + CatalogStorageFormat, CatalogTablePartition, + ExternalCatalogWithListener, FunctionExpressionBuilder, 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} @@ -53,19 +56,23 @@ class CarbonHiveSessionCatalog( externalCatalog: HiveExternalCatalog, globalTempViewManager: GlobalTempViewManager, functionRegistry: FunctionRegistry, + tableFunctionRegistry: TableFunctionRegistry, sparkSession: SparkSession, conf: SQLConf, hadoopConf: Configuration, parser: ParserInterface, - functionResourceLoader: FunctionResourceLoader) + functionResourceLoader: FunctionResourceLoader, + functionExpressionBuilder: FunctionExpressionBuilder) extends HiveSessionCatalog ( () => externalCatalog, () => globalTempViewManager, new HiveMetastoreCatalog(sparkSession), functionRegistry, + tableFunctionRegistry, hadoopConf, parser, - functionResourceLoader + functionResourceLoader, + functionExpressionBuilder ) with CarbonSessionCatalog { private lazy val carbonEnv = { @@ -144,7 +151,7 @@ class CarbonHiveSessionCatalog( */ class CarbonSessionStateBuilder(sparkSession: SparkSession, parentState: Option[SessionState] = None) - extends HiveSessionStateBuilder(sparkSession, parentState, Map.empty) { + extends HiveSessionStateBuilder(sparkSession, parentState) { override lazy val sqlParser: ParserInterface = new CarbonSparkSqlParser(conf, sparkSession) @@ -164,11 +171,13 @@ class CarbonSessionStateBuilder(sparkSession: SparkSession, externalCatalog, session.sharedState.globalTempViewManager, functionRegistry, + tableFunctionRegistry, sparkSession, conf, SessionState.newHadoopConf(session.sparkContext.hadoopConfiguration, conf), sqlParser, - resourceLoader) + resourceLoader, + HiveUDFExpressionBuilder) parentState.foreach(_.catalog.copyStateTo(catalog)) catalog } diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala similarity index 93% rename from integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala index fcfa54817cc..6e1b3ae5636 100644 --- a/integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/CarbonSqlAstBuilder.scala @@ -19,11 +19,11 @@ 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, CreateTableContext, CreateTableLikeContext} +import org.apache.spark.sql.catalyst.parser.SqlBaseParser.{AddTableColumnsContext, CreateTableContext} import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.SparkSqlAstBuilder import org.apache.spark.sql.internal.SQLConf -import org.apache.spark.sql.parser.{CarbonHelperSqlAstBuilder, CarbonSpark2SqlParser, CarbonSparkSqlParserUtil} +import org.apache.spark.sql.parser.{CarbonSpark2SqlParser, CarbonSparkSqlParserUtil} class CarbonSqlAstBuilder(conf: SQLConf, parser: CarbonSpark2SqlParser, sparkSession: SparkSession) extends SparkSqlAstBuilder with SqlAstBuilderHelper { diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala similarity index 92% rename from integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala index 08c053825ff..04dc236e83f 100644 --- a/integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/SqlAstBuilderHelper.scala @@ -17,21 +17,16 @@ package org.apache.spark.sql.hive -import java.util - -import scala.collection.JavaConverters._ - -import org.apache.spark.sql.CarbonToSparkAdapter 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.catalyst.parser.SqlBaseParser.{AddTableColumnsContext, CreateTableContext} +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan 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} +import org.apache.spark.sql.types.DecimalType trait SqlAstBuilderHelper extends SparkSqlAstBuilder { diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala index a3e3b41c109..27983d75701 100644 --- a/integration/spark/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonHiveCommands.scala @@ -22,6 +22,7 @@ import org.apache.spark.sql.{CarbonEnv, Row, SparkSession} import org.apache.spark.sql.catalyst.TableIdentifier import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException} import org.apache.spark.sql.catalyst.expressions.Attribute +import org.apache.spark.sql.catalyst.plans.logical.LogicalPlan import org.apache.spark.sql.execution.command._ import org.apache.spark.sql.execution.command.table.CarbonDropTableCommand @@ -86,6 +87,11 @@ case class CarbonDropDatabaseCommand(command: DropDatabaseCommand) CarbonUtil.dropDatabaseDirectory(carbonDatabaseLocation) rows } + + override protected def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]) + : LogicalPlan = { + this + } } case class CarbonSetCommand(command: SetCommand) @@ -105,6 +111,10 @@ case class CarbonSetCommand(command: SetCommand) override protected def opName: String = "SET" + override protected def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]) + : LogicalPlan = { + this + } } object CarbonSetCommand { diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala b/integration/spark/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala similarity index 92% rename from integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala index 784e8c7227d..14c648792b1 100644 --- a/integration/spark/src/main/spark3.1/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/hive/execution/command/CarbonResetCommand.scala @@ -29,6 +29,11 @@ case class CarbonResetCommand() CarbonEnv.getInstance(sparkSession).carbonSessionInfo.getSessionParams.clear() ResetCommand(None).run(sparkSession) } + + override protected def withNewChildrenInternal(newChildren: IndexedSeq[LogicalPlan]) + : LogicalPlan = { + this + } } /** diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala b/integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala similarity index 100% rename from integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonExtensionSqlParser.scala diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala b/integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala similarity index 94% rename from integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala rename to integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala index 49978d4e3c2..61efd610dc9 100644 --- a/integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParser.scala @@ -16,12 +16,10 @@ */ 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.ParserUtils.operationNotAllowed +import org.apache.spark.sql.catalyst.parser.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} @@ -81,22 +79,22 @@ class CarbonSparkSqlParser(conf: SQLConf, sparkSession: SparkSession) extends Sp class CarbonHelperSqlAstBuilder(conf: SQLConf, parser: CarbonSpark2SqlParser, sparkSession: SparkSession) - extends SparkSqlAstBuilderWrapper(conf) { + extends SparkSqlAstBuilder { /** * 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) + override def visitPropertyKeyValues(ctx: PropertyListContext): Map[String, String] = { + val props = visitPropertyKeyValues(ctx) CarbonSparkSqlParserUtil.visitPropertyKeyValues(ctx, props) } - def getPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] = { + def getPropertyKeyValues(ctx: PropertyListContext): Map[String, String] = { Option(ctx).map(visitPropertyKeyValues) .getOrElse(Map.empty) } def createCarbonTable(createTableTuple: (CreateTableHeaderContext, SkewSpecContext, - BucketSpecContext, PartitionFieldListContext, ColTypeListContext, TablePropertyListContext, + BucketSpecContext, PartitionFieldListContext, ColTypeListContext, PropertyListContext, LocationSpecContext, Option[String], TerminalNode, QueryContext, String)): LogicalPlan = { val (tableHeader, skewSpecContext, @@ -111,7 +109,8 @@ class CarbonHelperSqlAstBuilder(conf: SQLConf, provider) = createTableTuple val (tableIdent, temp, ifNotExists, external) = visitCreateTableHeader(tableHeader) - val tableIdentifier = CarbonToSparkAdapter.getTableIdentifier(tableIdent) + val parts = visitMultipartIdentifier(tableIdent.multipartIdentifier()) + val tableIdentifier = CarbonToSparkAdapter.getTableIdentifier(parts) val cols: Seq[StructField] = Option(columns).toSeq.flatMap(visitColTypeList) val colNames: Seq[String] = CarbonSparkSqlParserUtil .validateCreateTableReqAndGetColumns(tableHeader, diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala b/integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala index d4b313c2dd8..f653adad6fc 100644 --- a/integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/parser/CarbonSparkSqlParserUtil.scala @@ -339,7 +339,7 @@ object CarbonSparkSqlParserUtil { * @param props Map of table property list * @return Map of transformed table property. */ - def visitPropertyKeyValues(ctx: TablePropertyListContext, + def visitPropertyKeyValues(ctx: PropertyListContext, props: Map[String, String]): Map[String, String] = { val badKeys = props.filter { case (_, v) => v == null }.keys if (badKeys.nonEmpty) { @@ -436,7 +436,7 @@ object CarbonSparkSqlParserUtil { case Some(value) => val result = value.children.get(1).getText if (result.equalsIgnoreCase("by")) { - value.storageHandler().STRING().getSymbol.getText + value.storageHandler().stringLit().STRING_LITERAL().getSymbol.getText } else if (result.equalsIgnoreCase("as") && value.children.size() > 1) { value.children.get(2).getText } else { diff --git a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/hive/CarbonInternalMetastore.scala b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/hive/CarbonInternalMetastore.scala index 4144963ff08..58cafc2824e 100644 --- a/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/hive/CarbonInternalMetastore.scala +++ b/integration/spark/src/main/scala/org/apache/spark/sql/secondaryindex/hive/CarbonInternalMetastore.scala @@ -249,9 +249,9 @@ object CarbonInternalMetastore { } } } catch { - case e: Exception => + case _: Exception => // In case of creating a table, hive table will not be available. - LOGGER.error(e.getMessage) + // LOGGER.error(e.getMessage) } } // add cg and fg index info to table properties diff --git a/integration/spark/src/main/spark2.3/com/databricks/spark/avro/AvroWriter.scala b/integration/spark/src/main/spark2.3/com/databricks/spark/avro/AvroWriter.scala deleted file mode 100644 index 8de31bec27f..00000000000 --- a/integration/spark/src/main/spark2.3/com/databricks/spark/avro/AvroWriter.scala +++ /dev/null @@ -1,51 +0,0 @@ -/* - * 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 com.databricks.spark.avro - -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.Row -import org.apache.spark.sql.execution.datasources.OutputWriterFactory - -/** - * This class is to get the avro writer from databricks avro module, as its not present in spark2.3 - * and spark-avro module is included in spark project from spark-2.4. So for spark-2.4, we use Avro - * writer from spark project. - */ -object AvroWriter { - - def getWriter(spark: org.apache.spark.sql.SparkSession, - job: org.apache.hadoop.mapreduce.Job, - dataSchema: org.apache.spark.sql.types.StructType, - options: scala.Predef.Map[scala.Predef.String, scala.Predef.String] = Map.empty) - : OutputWriterFactory = { - new DefaultSource().prepareWrite(spark, job, - options, dataSchema) - } -} - -/** - * This reads the avro files from the given path and return the RDD[Row] - */ -object AvroReader { - - def readAvro(spark: org.apache.spark.sql.SparkSession, deltaPath: String): RDD[Row] = { - spark.sparkContext - .hadoopConfiguration - .set("avro.mapred.ignore.inputs.without.extension", "false") - spark.read.avro(deltaPath).rdd - } -} diff --git a/integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonBoundReference.scala b/integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonBoundReference.scala deleted file mode 100644 index 3b185c3f783..00000000000 --- a/integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonBoundReference.scala +++ /dev/null @@ -1,45 +0,0 @@ -/* - * 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) - extends LeafExpression with NamedExpression with CodegenFallback { - - type EvaluatedType = Any - - override def toString: String = s"input[" + colExp.getColIndex + "]" - - override def eval(input: InternalRow): Any = input.get(colExp.getColIndex, dataType) - - override def name: String = colExp.getColumnName - - override def toAttribute: Attribute = throw new UnsupportedOperationException - - override def exprId: ExprId = throw new UnsupportedOperationException - - override def qualifier: Option[String] = null - - override def newInstance(): NamedExpression = throw new UnsupportedOperationException -} diff --git a/integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapter.scala b/integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapter.scala deleted file mode 100644 index 2c3483bdf53..00000000000 --- a/integration/spark/src/main/spark2.3/org/apache/spark/sql/CarbonToSparkAdapter.scala +++ /dev/null @@ -1,230 +0,0 @@ -/* - * 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 -import org.apache.spark.scheduler.{SparkListener, SparkListenerApplicationEnd} -import org.apache.spark.sql.carbondata.execution.datasources.CarbonFileIndexReplaceRule -import org.apache.spark.sql.catalyst.catalog.{CatalogStorageFormat, SessionCatalog} -import org.apache.spark.sql.catalyst.expressions.{Alias, Attribute, AttributeReference, AttributeSet, Expression, ExprId, NamedExpression, ScalaUDF, SubqueryExpression} -import org.apache.spark.sql.catalyst.expressions.codegen.ExprCode -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.{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.ThreadLocalSessionInfo -import org.apache.carbondata.geo.{InPolygonJoinUDF, ToRangeListAsStringUDF} - -object CarbonToSparkAdapter extends SparkVersionAdapter { - - def createFilePartition(index: Int, files: ArrayBuffer[PartitionedFile]): FilePartition = { - FilePartition(index, files.toArray.toSeq) - } - - 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 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(name: String, dataType: DataType, nullable: Boolean, - metadata: Metadata, exprId: ExprId, qualifier: Option[String], - attrRef : NamedExpression = null): AttributeReference = { - AttributeReference( - name, - dataType, - nullable, - metadata)(exprId, qualifier) - } - - def createAttributeReference(attr: AttributeReference, - attrName: String, - newSubsume: String): AttributeReference = { - AttributeReference(attrName, attr.dataType)( - exprId = attr.exprId, - qualifier = Some(newSubsume)) - } - - def getTheLastQualifier(attribute: Attribute): String = { - attribute.qualifier.head - } - - def getOutput(subQueryAlias: SubqueryAlias): Seq[Attribute] = { - subQueryAlias.output - } - - def createScalaUDF(s: ScalaUDF, reference: AttributeReference): ScalaUDF = { - ScalaUDF(s.function, s.dataType, Seq(reference), s.inputTypes) - } - - def createRangeListScalaUDF(toRangeListUDF: ToRangeListAsStringUDF, - dataType: StringType.type, - children: Seq[Expression], - inputTypes: Seq[DataType]): ScalaUDF = { - ScalaUDF(toRangeListUDF, - dataType, - children, - inputTypes, - Some("ToRangeListAsString")) - } - - def getTransformedPolygonJoinUdf(scalaUdf: ScalaUDF, - udfChildren: Seq[Expression], - polygonJoinUdf: InPolygonJoinUDF): ScalaUDF = { - ScalaUDF(polygonJoinUdf, - scalaUdf.dataType, - udfChildren, - scalaUdf.inputTypes :+ scalaUdf.inputTypes.head, - scalaUdf.udfName, - scalaUdf.nullable, - scalaUdf.udfDeterministic) - } - - def createExprCode(code: String, isNull: String, value: String, dataType: DataType = null - ): ExprCode = { - ExprCode(code, isNull, value) - } - - def createAliasRef(child: Expression, - name: String, - exprId: ExprId = NamedExpression.newExprId, - qualifier: Option[String] = None, - explicitMetadata: Option[Metadata] = None) : Alias = { - - Alias(child, name)(exprId, qualifier, explicitMetadata) - } - - // Create the aliases using two plan outputs mappings. - def createAliases(mappings: Seq[(NamedExpression, NamedExpression)]): Seq[NamedExpression] = { - mappings.map{ case (o1, o2) => - o2 match { - case al: Alias if o1.name == o2.name && o1.exprId != o2.exprId => - Alias(al.child, o1.name)(exprId = o1.exprId) - case other => - if (o1.name != o2.name || o1.exprId != o2.exprId) { - Alias(o2, o1.name)(exprId = o1.exprId) - } else { - o2 - } - } - } - } - - /** - * 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], - partitionFilter: 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 getHiveExternalCatalog(sparkSession: SparkSession): HiveExternalCatalog = { - sparkSession.sessionState.catalog.externalCatalog.asInstanceOf[HiveExternalCatalog] - } -} - -class CarbonOptimizer( - session: SparkSession, - catalog: SessionCatalog, - optimizer: Optimizer) extends Optimizer(catalog) { - - private lazy val mvRules = Seq(Batch("Materialized View Optimizers", Once, - Seq(new MVRewriteRule(session)): _*)) - - private lazy val iudRule = Batch("IUD Optimizers", fixedPoint, - Seq(new CarbonIUDRule(), new CarbonUDFTransformRule(), new CarbonFileIndexReplaceRule()): _*) - - private lazy val secondaryIndexRule = Batch("SI Optimizers", Once, - Seq(new CarbonSITransformationRule(session)): _*) - - override def batches: Seq[Batch] = { - mvRules ++ convertedBatch() :+ iudRule :+ secondaryIndexRule - } - - def convertedBatch(): Seq[Batch] = { - optimizer.batches.map { batch => - Batch( - batch.name, - batch.strategy match { - case optimizer.Once => - Once - case _: optimizer.FixedPoint => - fixedPoint - }, - batch.rules: _* - ) - } - } -} diff --git a/integration/spark/src/main/spark2.3/org/apache/spark/sql/SparkSqlAdapter.scala b/integration/spark/src/main/spark2.3/org/apache/spark/sql/SparkSqlAdapter.scala deleted file mode 100644 index eb3c5fdc834..00000000000 --- a/integration/spark/src/main/spark2.3/org/apache/spark/sql/SparkSqlAdapter.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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.scheduler.{SparkListener, SparkListenerApplicationEnd} -import org.apache.spark.sql.catalyst.TableIdentifier -import org.apache.spark.sql.catalyst.expressions.{Attribute, Expression} -import org.apache.spark.sql.execution.FileSourceScanExec -import org.apache.spark.sql.execution.datasources.HadoopFsRelation -import org.apache.spark.sql.types.StructType - -import org.apache.carbondata.core.util.ThreadLocalSessionInfo - -object SparkSqlAdapter { - - def initSparkSQL(): Unit = { - } - - def getScanForSegments( - @transient relation: HadoopFsRelation, - output: Seq[Attribute], - outputSchema: StructType, - partitionFilters: Seq[Expression], - dataFilters: Seq[Expression], - tableIdentifier: Option[TableIdentifier] - ): FileSourceScanExec = { - FileSourceScanExec( - relation, - output, - outputSchema, - partitionFilters, - dataFilters, - tableIdentifier) - } -} diff --git a/integration/spark/src/main/spark2.3/org/apache/spark/sql/avro/AvroFileFormatFactory.scala b/integration/spark/src/main/spark2.3/org/apache/spark/sql/avro/AvroFileFormatFactory.scala deleted file mode 100644 index 616f052bca8..00000000000 --- a/integration/spark/src/main/spark2.3/org/apache/spark/sql/avro/AvroFileFormatFactory.scala +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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 - -import com.databricks.spark.avro.{AvroReader, AvroWriter} -import org.apache.spark.rdd.RDD -import org.apache.spark.sql.Row -import org.apache.spark.sql.execution.datasources.OutputWriterFactory - -object AvroFileFormatFactory { - - /** - * return the avro writer to write the avro files - * @return avro writer - */ - def getAvroWriter(spark: org.apache.spark.sql.SparkSession, - job: org.apache.hadoop.mapreduce.Job, - dataSchema: org.apache.spark.sql.types.StructType, - options: scala.Predef.Map[scala.Predef.String, scala.Predef.String] = Map.empty) - : OutputWriterFactory = { - AvroWriter.getWriter(spark, job, dataSchema, options) - } - - /** - * Reads the avro files present at the given path - * @param deltaPath path to read the avro files from. - * @return RDD[Row] - */ - def readAvro(spark: org.apache.spark.sql.SparkSession, deltaPath: String): RDD[Row] = { - spark.sparkContext - .hadoopConfiguration - .set("avro.mapred.ignore.inputs.without.extension", "false") - AvroReader.readAvro(spark, deltaPath) - } -} diff --git a/integration/spark/src/main/spark2.3/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala b/integration/spark/src/main/spark2.3/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala deleted file mode 100644 index 9a54f8a5886..00000000000 --- a/integration/spark/src/main/spark2.3/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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 { - def createDataSource(catalogTable: CatalogTable, - ignoreIfExists: Boolean, - sparkSession: SparkSession): Seq[Row] = { - org.apache.spark.sql.execution.command. - CreateDataSourceTableCommand(catalogTable, ignoreIfExists).run(sparkSession) - } -} diff --git a/integration/spark/src/main/spark2.3/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala b/integration/spark/src/main/spark2.3/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala deleted file mode 100644 index b08f1060814..00000000000 --- a/integration/spark/src/main/spark2.3/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala +++ /dev/null @@ -1,189 +0,0 @@ -/* - * 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, 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.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 carbon table 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( - externalCatalog: HiveExternalCatalog, - globalTempViewManager: GlobalTempViewManager, - functionRegistry: FunctionRegistry, - sparkSession: SparkSession, - conf: SQLConf, - hadoopConf: Configuration, - parser: ParserInterface, - functionResourceLoader: FunctionResourceLoader) - extends HiveSessionCatalog ( - externalCatalog, - globalTempViewManager, - new HiveMetastoreCatalog(sparkSession), - functionRegistry, - conf, - hadoopConf, - parser, - functionResourceLoader - ) with CarbonSessionCatalog { - - private lazy val carbonEnv = { - val env = new CarbonEnv - env.init(sparkSession) - env - } - - /** - * return's the carbonEnv instance - * @return - */ - override def getCarbonEnv() : CarbonEnv = { - carbonEnv - } - - // Initialize all listeners to the Operation bus. - CarbonEnv.init - - override def lookupRelation(name: TableIdentifier): LogicalPlan = { - var rtnRelation = super.lookupRelation(name) - val isRelationRefreshed = - CarbonSessionUtil.refreshRelationAndSetStats(rtnRelation, name)(sparkSession) - if (isRelationRefreshed) { - rtnRelation = super.lookupRelation(name) - // Reset the stats after lookup. - CarbonSessionUtil.refreshRelationAndSetStats(rtnRelation, name)(sparkSession) - } - rtnRelation - } - - override def getCachedPlan(t: QualifiedTableName, - c: Callable[LogicalPlan]): LogicalPlan = { - val plan = super.getCachedPlan(t, c) - CarbonSessionUtil.updateCachedPlan(plan) - } - - /** - * returns hive client from HiveExternalCatalog - * - * @return - */ - override def getClient(): org.apache.spark.sql.hive.client.HiveClient = { - CarbonSessionCatalogUtil.getClient(sparkSession) - } - - /** - * This is alternate way of getting partition information. It first fetches all partitions from - * hive and then apply filter instead of querying hive along with filters. - * @param partitionFilters - * @param sparkSession - * @param carbonTable - * @return - */ - override def getPartitionsAlternate(partitionFilters: Seq[Expression], - sparkSession: SparkSession, carbonTable: CarbonTable): Seq[CatalogTablePartition] = { - CarbonSessionCatalogUtil.getPartitionsAlternate(partitionFilters, sparkSession, carbonTable) - } - - /** - * Update the storage format with new location information - */ - override def updateStorageLocation( - path: Path, - storage: CatalogStorageFormat, - newTableName: String, - dbName: String): CatalogStorageFormat = { - CarbonSessionCatalogUtil.updateStorageLocation(path, storage, newTableName, dbName) - } -} - -/** - * Session state implementation to override sql parser and adding strategies - * - * @param sparkSession - */ -class CarbonSessionStateBuilder(sparkSession: SparkSession, - parentState: Option[SessionState] = None) - extends HiveSessionStateBuilder(sparkSession, parentState) { - - override lazy val sqlParser: ParserInterface = new CarbonSparkSqlParser(conf, sparkSession) - - experimentalMethods.extraStrategies = - Seq(StreamingTableStrategy, DMLStrategy, DDLStrategy, CarbonSourceStrategy) - experimentalMethods.extraOptimizations = Seq(new CarbonIUDRule, new CarbonUDFTransformRule) - - /** - * Internal catalog for managing table and database states. - */ - /** - * Create a [[CarbonSessionStateBuilder]]. - */ - override protected lazy val catalog: CarbonHiveSessionCatalog = { - val catalog = new CarbonHiveSessionCatalog( - externalCatalog, - session.sharedState.globalTempViewManager, - functionRegistry, - sparkSession, - conf, - SessionState.newHadoopConf(session.sparkContext.hadoopConfiguration, conf), - sqlParser, - resourceLoader) - parentState.foreach(_.catalog.copyStateTo(catalog)) - catalog - } - - private def externalCatalog: HiveExternalCatalog = - session.sharedState.externalCatalog.asInstanceOf[HiveExternalCatalog] - - /** - * Create a Hive aware resource loader. - */ - override protected lazy val resourceLoader: HiveSessionResourceLoader = { - val client: HiveClient = externalCatalog.client.newSession() - new HiveSessionResourceLoader(session, client) - } - - override protected def analyzer: Analyzer = - new CarbonAnalyzer(catalog, conf, sparkSession, super.analyzer) -} - diff --git a/integration/spark/src/main/spark2.4/org/apache/spark/sql/CarbonToSparkAdapter.scala b/integration/spark/src/main/spark2.4/org/apache/spark/sql/CarbonToSparkAdapter.scala deleted file mode 100644 index 89212a6d985..00000000000 --- a/integration/spark/src/main/spark2.4/org/apache/spark/sql/CarbonToSparkAdapter.scala +++ /dev/null @@ -1,272 +0,0 @@ -/* - * 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 -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, SessionCatalog} -import org.apache.spark.sql.catalyst.expressions._ -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.parser.SqlBaseParser._ -import org.apache.spark.sql.catalyst.plans.logical.{LogicalPlan, SubqueryAlias} -import org.apache.spark.sql.catalyst.rules.Rule -import org.apache.spark.sql.execution.command._ -import org.apache.spark.sql.execution.datasources.{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.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 = { - ScalaUDF(s.function, s.dataType, Seq(reference), s.inputsNullSafe, s.inputTypes) - } - - def createRangeListScalaUDF(toRangeListUDF: ToRangeListAsStringUDF, - dataType: StringType.type, - children: Seq[Expression], - inputTypes: Seq[DataType]): ScalaUDF = { - val inputsNullSafe: Seq[Boolean] = Seq(true, true, true) - ScalaUDF(toRangeListUDF, - dataType, - children, - inputsNullSafe, - inputTypes, - Some("ToRangeListAsString")) - } - - def getTransformedPolygonJoinUdf(scalaUdf: ScalaUDF, - udfChildren: Seq[Expression], - polygonJoinUdf: InPolygonJoinUDF): ScalaUDF = { - ScalaUDF(polygonJoinUdf, - scalaUdf.dataType, - udfChildren, - scalaUdf.inputsNullSafe, - scalaUdf.inputTypes :+ scalaUdf.inputTypes.head, - scalaUdf.udfName) - } - - 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) - } - - // Create the aliases using two plan outputs mappings. - def createAliases(mappings: Seq[(NamedExpression, NamedExpression)]): Seq[NamedExpression] = { - mappings.map{ case (o1, o2) => - o2 match { - case al: Alias if o1.name == o2.name && o1.exprId != o2.exprId => - Alias(al.child, o1.name)(exprId = o1.exprId) - case other => - if (o1.name != o2.name || o1.exprId != o2.exprId) { - Alias(o2, o1.name)(exprId = o1.exprId) - } else { - o2 - } - } - } - } - - def getTheLastQualifier(attribute: Attribute): String = { - attribute.qualifier.reverse.head - } - - - /** - * 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], - partitionFilter: 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.name.identifier) - subQueryAlias.child.output.map(_.withQualifier(newAlias)) - } - - def getHiveExternalCatalog(sparkSession: SparkSession): HiveExternalCatalog = { - sparkSession.sessionState.catalog.externalCatalog - .asInstanceOf[ExternalCatalogWithListener] - .unwrapped - .asInstanceOf[HiveExternalCatalog] - } -} - -class CarbonOptimizer( - session: SparkSession, - catalog: SessionCatalog, - optimizer: Optimizer) extends Optimizer(catalog) { - - private lazy val mvRules = Seq(Batch("Materialized View Optimizers", Once, - Seq(new MVRewriteRule(session)): _*)) - - private lazy val iudRule = Batch("IUD Optimizers", fixedPoint, - Seq(new CarbonIUDRule(), new CarbonUDFTransformRule(), new CarbonFileIndexReplaceRule()): _*) - - private lazy val secondaryIndexRule = Batch("SI Optimizers", Once, - Seq(new CarbonSITransformationRule(session)): _*) - - override def defaultBatches: Seq[Batch] = { - mvRules ++ convertedBatch() :+ iudRule :+ secondaryIndexRule - } - - def convertedBatch(): Seq[Batch] = { - optimizer.batches.map { batch => - Batch( - batch.name, - batch.strategy match { - case optimizer.Once => - Once - case _: optimizer.FixedPoint => - fixedPoint - }, - batch.rules: _* - ) - } - } -} diff --git a/integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala b/integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala deleted file mode 100644 index bace849cc37..00000000000 --- a/integration/spark/src/main/spark2.4/org/apache/spark/sql/hive/CarbonSessionStateBuilder.scala +++ /dev/null @@ -1,198 +0,0 @@ -/* - * 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.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( - externalCatalog: HiveExternalCatalog, - globalTempViewManager: GlobalTempViewManager, - functionRegistry: FunctionRegistry, - sparkSession: SparkSession, - conf: SQLConf, - hadoopConf: Configuration, - parser: ParserInterface, - functionResourceLoader: FunctionResourceLoader) - extends HiveSessionCatalog ( - () => externalCatalog, - () => globalTempViewManager, - new HiveMetastoreCatalog(sparkSession), - functionRegistry, - conf, - hadoopConf, - parser, - functionResourceLoader - ) with CarbonSessionCatalog { - - private lazy val carbonEnv = { - val env = new CarbonEnv - env.init(sparkSession) - env - } - - /** - * return's the carbonEnv instance - * @return - */ - override def getCarbonEnv() : CarbonEnv = { - carbonEnv - } - - // Initialize all listeners to the Operation bus. - CarbonEnv.init - - override def lookupRelation(name: TableIdentifier): LogicalPlan = { - var rtnRelation = super.lookupRelation(name) - val isRelationRefreshed = - CarbonSessionUtil.refreshRelationAndSetStats(rtnRelation, name)(sparkSession) - if (isRelationRefreshed) { - rtnRelation = super.lookupRelation(name) - // Reset the stats after lookup. - CarbonSessionUtil.refreshRelationAndSetStats(rtnRelation, name)(sparkSession) - } - rtnRelation - } - - override def getCachedPlan(t: QualifiedTableName, - c: Callable[LogicalPlan]): LogicalPlan = { - val plan = super.getCachedPlan(t, c) - CarbonSessionUtil.updateCachedPlan(plan) - } - - /** - * returns hive client from HiveExternalCatalog - * - * @return - */ - override def getClient(): org.apache.spark.sql.hive.client.HiveClient = { - CarbonSessionCatalogUtil.getClient(sparkSession) - } - - /** - * This is alternate way of getting partition information. It first fetches all partitions from - * hive and then apply filter instead of querying hive along with filters. - * @param partitionFilters - * @param sparkSession - * @param carbonTable - * @return - */ - override def getPartitionsAlternate(partitionFilters: Seq[Expression], - sparkSession: SparkSession, carbonTable: CarbonTable): Seq[CatalogTablePartition] = { - CarbonSessionCatalogUtil.getPartitionsAlternate(partitionFilters, sparkSession, carbonTable) - } - - /** - * Update the storageformat with new location information - */ - override def updateStorageLocation( - path: Path, - storage: CatalogStorageFormat, - newTableName: String, - dbName: String): CatalogStorageFormat = { - CarbonSessionCatalogUtil.updateStorageLocation(path, storage, newTableName, dbName) - } -} - -/** - * Session state implementation to override sql parser and adding strategies - * - * @param sparkSession - */ -class CarbonSessionStateBuilder(sparkSession: SparkSession, - parentState: Option[SessionState] = None) - extends HiveSessionStateBuilder(sparkSession, parentState) { - - override lazy val sqlParser: ParserInterface = new CarbonSparkSqlParser(conf, sparkSession) - - experimentalMethods.extraStrategies = - Seq(StreamingTableStrategy, DMLStrategy, DDLStrategy, CarbonSourceStrategy) - experimentalMethods.extraOptimizations = Seq(new CarbonIUDRule, - new CarbonUDFTransformRule) - - /** - * Internal catalog for managing table and database states. - */ - /** - * Create a [[CarbonSessionStateBuilder]]. - */ - override protected lazy val catalog: CarbonHiveSessionCatalog = { - val catalog = new CarbonHiveSessionCatalog( - externalCatalog, - session.sharedState.globalTempViewManager, - functionRegistry, - sparkSession, - conf, - SessionState.newHadoopConf(session.sparkContext.hadoopConfiguration, conf), - sqlParser, - resourceLoader) - parentState.foreach(_.catalog.copyStateTo(catalog)) - catalog - } - - private def externalCatalog: HiveExternalCatalog = - session - .sharedState - .externalCatalog - .asInstanceOf[ExternalCatalogWithListener] - .unwrapped - .asInstanceOf[HiveExternalCatalog] - - /** - * Create a Hive aware resource loader. - */ - override protected lazy val resourceLoader: HiveSessionResourceLoader = { - val client: HiveClient = externalCatalog.client.newSession() - new HiveSessionResourceLoader(session, () => client) - } - - override protected def analyzer: Analyzer = { - new CarbonAnalyzer(catalog, - conf, - sparkSession, - super.analyzer) - } -} diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkSqlAdapter.scala b/integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkSqlAdapter.scala deleted file mode 100644 index 8be562322f6..00000000000 --- a/integration/spark/src/main/spark3.1/org/apache/spark/sql/SparkSqlAdapter.scala +++ /dev/null @@ -1,49 +0,0 @@ -/* - * 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, - output: Seq[Attribute], - outputSchema: StructType, - partitionFilters: Seq[Expression], - dataFilters: Seq[Expression], - tableIdentifier: Option[TableIdentifier] - ): FileSourceScanExec = { - FileSourceScanExec( - relation, - output, - outputSchema, - partitionFilters, - None, - None, - dataFilters, - tableIdentifier) - } -} diff --git a/integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/SparkSqlAstBuilderWrapper.scala b/integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/SparkSqlAstBuilderWrapper.scala deleted file mode 100644 index 7a576b16d8a..00000000000 --- a/integration/spark/src/main/spark3.1/org/apache/spark/sql/parser/SparkSqlAstBuilderWrapper.scala +++ /dev/null @@ -1,30 +0,0 @@ -/* - * 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 { - - def visitPropertyKeyValues(ctx: TablePropertyListContext): Map[String, String] -} diff --git a/integration/spark/src/test/resources/datawithoutheader.csv b/integration/spark/src/test/resources/datawithoutheader.csv deleted file mode 100644 index 762ec849168..00000000000 --- a/integration/spark/src/test/resources/datawithoutheader.csv +++ /dev/null @@ -1,10 +0,0 @@ -11,arvind,SE,17-01-2007,1,developer,10,network,928478,17-02-2007,29-11-2016,96,96,5040 -12,krithin,SSE,29-05-2008,1,developer,11,protocol,928378,29-06-2008,30-12-2016,85,95,7124 -13,madhan,TPL,07-07-2009,2,tester,10,network,928478,07-08-2009,30-12-2016,88,99,9054 -14,anandh,SA,29-12-2010,3,manager,11,protocol,928278,29-01-2011,29-06-2016,77,92,11248 -15,ayushi,SSA,09-11-2011,1,developer,12,security,928375,09-12-2011,29-05-2016,99,91,13245 -16,pramod,SE,14-10-2012,1,developer,13,configManagement,928478,14-11-2012,29-12-2016,86,93,5040 -17,gawrav,PL,22-09-2013,2,tester,12,security,928778,22-10-2013,15-11-2016,78,97,9574 -18,sibi,TL,15-08-2014,2,tester,14,Learning,928176,15-09-2014,29-05-2016,84,98,7245 -19,shivani,PL,12-05-2015,1,developer,10,network,928977,12-06-2015,12-11-2016,88,91,11254 -20,bill,PM,01-12-2015,3,manager,14,Learning,928479,01-01-2016,30-11-2016,75,94,13547 diff --git a/integration/spark/src/test/scala/org/apache/carbondata/view/rewrite/TestAllOperationsOnMV.scala b/integration/spark/src/test/scala/org/apache/carbondata/view/rewrite/TestAllOperationsOnMV.scala index 20586dab88b..9907f6eee06 100644 --- a/integration/spark/src/test/scala/org/apache/carbondata/view/rewrite/TestAllOperationsOnMV.scala +++ b/integration/spark/src/test/scala/org/apache/carbondata/view/rewrite/TestAllOperationsOnMV.scala @@ -47,6 +47,8 @@ class TestAllOperationsOnMV extends QueryTest with BeforeAndAfterEach { sql("drop table IF EXISTS testtable") sql("create table testtable(name string, c_code int, price int) STORED AS carbondata") sql("insert into table testtable select 'abc',21,2000") + sql("drop table if exists dm1") + sql("drop view if exists dm1") sql("drop materialized view if exists dm1") sql("create materialized view dm1 with deferred refresh as select name,sum(price) " + "from maintable group by name") diff --git a/mv/plan/pom.xml b/mv/plan/pom.xml index 42353658d4d..a61330dca94 100644 --- a/mv/plan/pom.xml +++ b/mv/plan/pom.xml @@ -142,130 +142,5 @@ - - - spark-2.3 - - 2.3 - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - src/main/spark3.1 - src/main/spark2.4 - - - - - org.codehaus.mojo - build-helper-maven-plugin - 3.0.0 - - - add-source - generate-sources - - add-source - - - - src/main/spark2.3 - src/main/common2.3and2.4 - - - - - - - - - - spark-2.4 - - true - - - 2.4 - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - src/main/spark3.1 - src/main/spark2.3 - - - - - org.codehaus.mojo - build-helper-maven-plugin - 3.0.0 - - - add-source - generate-sources - - add-source - - - - src/main/spark2.4 - src/main/common2.3and2.4 - - - - - - - - - - spark-3.1 - - 3.1 - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - src/main/spark2.3 - src/main/spark2.4 - src/main/common2.3and2.4 - - - - - org.codehaus.mojo - build-helper-maven-plugin - 3.0.0 - - - add-source - generate-sources - - add-source - - - - src/main/spark3.1 - - - - - - - - - diff --git a/mv/plan/src/main/common2.3and2.4/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala b/mv/plan/src/main/common2.3and2.4/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala deleted file mode 100644 index 78002f1198a..00000000000 --- a/mv/plan/src/main/common2.3and2.4/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala +++ /dev/null @@ -1,137 +0,0 @@ -/* - * 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) - 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) => - Some(left, right, joinType, condition, None) - case _ => None - } - } -} - -object MatchAggregateExpression { - def unapply(expr : AggregateExpression): Option[(AggregateFunction, AggregateMode, Boolean, - Option[Expression], ExprId)] = { - expr match { - case j@AggregateExpression(aggregateFunction, mode, isDistinct, resultId) => - Some(aggregateFunction, mode, isDistinct, None, resultId) - case _ => None - } - } -} diff --git a/mv/plan/src/main/scala/org/apache/carbondata/mv/expressions/modular/subquery.scala b/mv/plan/src/main/scala/org/apache/carbondata/mv/expressions/modular/subquery.scala index 43551bba3c2..1d8c72e8ecd 100644 --- a/mv/plan/src/main/scala/org/apache/carbondata/mv/expressions/modular/subquery.scala +++ b/mv/plan/src/main/scala/org/apache/carbondata/mv/expressions/modular/subquery.scala @@ -39,16 +39,10 @@ abstract class ModularSubquery( override def withNewPlan(plan: ModularPlan): ModularSubquery - override def semanticEquals(o: Expression): Boolean = { - o match { - case p: ModularSubquery => - this.getClass.getName.equals(p.getClass.getName) && plan.sameResult(p.plan) && - children.length == p.children.length && - children.zip(p.children).forall(p => p._1.semanticEquals(p._2)) - case _ => false - } + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[Expression]): Expression = { + throw new UnsupportedOperationException() } - def canonicalize(attrs: AttributeSeq): ModularSubquery = { // Normalize the outer references in the subquery plan. val normalizedPlan = plan.transformAllExpressions { diff --git a/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/AggregatePushDown.scala b/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/AggregatePushDown.scala index cbcd4248466..d03c7c0064e 100644 --- a/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/AggregatePushDown.scala +++ b/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/AggregatePushDown.scala @@ -128,7 +128,7 @@ trait AggregatePushDown { // self: ModularPlan => } else { Map.empty[Int, (NamedExpression, Seq[NamedExpression])] } - case sum@MatchAggregateExpression(Sum(cast@MatchCast(expr, dataType)), _, false, _, _) => + case sum@MatchAggregateExpression(Sum(cast@MatchCast(expr), _), _, false, _, _) => val tAttr = selAliasMap.get(expr.asInstanceOf[Attribute]).getOrElse(expr) .asInstanceOf[Attribute] if (fact.outputSet.contains(tAttr)) { @@ -190,7 +190,7 @@ trait AggregatePushDown { // self: ModularPlan => } else { Map.empty[Int, (NamedExpression, Seq[NamedExpression])] } - case avg@MatchAggregateExpression(Average(cast@MatchCast(expr, dataType)), _, false, _, _) => + case avg@MatchAggregateExpression(Average(cast@MatchCast(expr, _), _), _, false, _, _) => val tAttr = selAliasMap.get(expr.asInstanceOf[Attribute]).getOrElse(expr) .asInstanceOf[Attribute] if (fact.outputSet.contains(tAttr)) { diff --git a/mv/plan/src/main/spark2.4/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala b/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala similarity index 85% rename from mv/plan/src/main/spark2.4/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala rename to mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala index d1c9d8ab0d0..2bd1f6b639d 100644 --- a/mv/plan/src/main/spark2.4/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala +++ b/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala @@ -23,13 +23,13 @@ import org.apache.spark.sql.types.{DataType, Metadata} object ExpressionHelper { def createReference( - name: String, - dataType: DataType, - nullable: Boolean, - metadata: Metadata, - exprId: ExprId, - qualifier: Option[String], - attrRef : NamedExpression = null): AttributeReference = { + 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) } @@ -38,7 +38,7 @@ object ExpressionHelper { child: Expression, name: String, exprId: ExprId, - qualifier: Option[String]) : Alias = { + qualifier: Option[String]): Alias = { val qf = if (qualifier.nonEmpty) Seq(qualifier.get) else Seq.empty Alias(child, name)(exprId, qf, None) } diff --git a/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularPlan.scala b/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularPlan.scala index e39ef7c96e3..04e2bfeaf79 100644 --- a/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularPlan.scala +++ b/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/ModularPlan.scala @@ -206,6 +206,11 @@ abstract class ModularPlan protected def preHarmonized: ModularPlan = { this } + + override protected def withNewChildrenInternal( + newChildren: IndexedSeq[ModularPlan]): ModularPlan = { + throw new UnsupportedOperationException() + } } object ModularPlan extends PredicateHelper { diff --git a/mv/plan/src/main/spark3.1/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala b/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala similarity index 90% rename from mv/plan/src/main/spark3.1/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala rename to mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala index e52efc9d688..fc34c6f1630 100644 --- a/mv/plan/src/main/spark3.1/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala +++ b/mv/plan/src/main/scala/org/apache/carbondata/mv/plans/modular/SparkVersionHelper.scala @@ -21,7 +21,7 @@ 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.optimizer._ 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 diff --git a/mv/plan/src/main/spark2.3/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala b/mv/plan/src/main/spark2.3/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala deleted file mode 100644 index 8fa9be4a593..00000000000 --- a/mv/plan/src/main/spark2.3/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala +++ /dev/null @@ -1,50 +0,0 @@ -/* - * 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, AttributeReference, Expression, ExprId, NamedExpression} -import org.apache.spark.sql.types.{DataType, Metadata} - -object ExpressionHelper { - - def createReference( - name: String, - dataType: DataType, - nullable: Boolean, - metadata: Metadata, - exprId: ExprId, - qualifier: Option[String], - attrRef : NamedExpression = null): AttributeReference = { - AttributeReference(name, dataType, nullable, metadata)(exprId, qualifier) - } - - def createAlias( - child: Expression, - name: String, - exprId: ExprId = NamedExpression.newExprId, - qualifier: Option[String] = None, - explicitMetadata: Option[Metadata] = None, - namedExpr : Option[NamedExpression] = None ) : Alias = { - Alias(child, name)(exprId, qualifier, explicitMetadata) - } - - def getTheLastQualifier(reference: AttributeReference): String = { - reference.qualifier.head - } - -} diff --git a/mv/plan/src/main/spark3.1/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala b/mv/plan/src/main/spark3.1/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala deleted file mode 100644 index 3814ccaaebb..00000000000 --- a/mv/plan/src/main/spark3.1/org/apache/carbondata/mv/plans/modular/ExpressionHelper.scala +++ /dev/null @@ -1,58 +0,0 @@ -/* - * 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.{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.spark.sql.types.{DataType, Metadata} -import scala.reflect.ClassTag - -import org.apache.carbondata.mv.plans.util.BirdcageOptimizer - -object ExpressionHelper { - - def createReference( - 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 createAlias( - child: Expression, - name: String, - exprId: ExprId, - qualifier: Option[String]): Alias = { - val qf = if (qualifier.nonEmpty) Seq(qualifier.get) else Seq.empty - Alias(child, name)(exprId, qf, None) - } - - def getTheLastQualifier(reference: AttributeReference): String = { - reference.qualifier.reverse.head - } - -} diff --git a/pom.xml b/pom.xml index 804b0c8f2f2..eb44cec55a7 100644 --- a/pom.xml +++ b/pom.xml @@ -126,17 +126,17 @@ 2.7.2 4.3.4 4.3-alpha1 - 2.11 - 2.11.12 + 2.12 + 2.12.8 compile - 2.4.5 - 2.4 + 3.5.1 + 3.5 4.8 compile compile ${basedir}/dev local[2] - 2.6.5 + 2.10.0 local localhost:8086 @@ -452,9 +452,6 @@ false ${basedir}/src/main/scala - ${basedir}/src/main/spark${spark.binary.version} - ${basedir}/src/main/common2.3and2.4 - ${basedir}/src/main/common2.4and3.1 ${basedir}/src/test/scala scalastyle-config.xml @@ -573,230 +570,6 @@ 4.5.2 - - spark-2.3 - - 2.3 - 2.3.4 - 2.11 - 2.11.8 - - - - - org.codehaus.mojo - flatten-maven-plugin - 1.2.2 - - ${project.build.directory} - - - - - flatten - process-resources - - flatten - - - - - flatten.clean - clean - - clean - - - - - - org.eluder.coveralls - coveralls-maven-plugin - 4.3.0 - - opPwqWW41vYppv6KISea3u1TJvE1ugJ5Y - UTF-8 - - ${basedir}/target/carbondata-coverage-report/carbondata-coverage-report.xml - - - - ${basedir}/common/src/main/java - ${basedir}/core/src/main/java - ${basedir}/geo/src/main/java - ${basedir}/geo/src/main/scala - ${basedir}/processing/src/main/java - ${basedir}/hadoop/src/main/java - ${basedir}/integration/spark/src/main/scala - ${basedir}/integration/spark/src/main/spark2.3 - ${basedir}/integration/spark/src/main/common2.3and2.4 - ${basedir}/integration/spark/src/main/java - ${basedir}/integration/hive/src/main/scala - ${basedir}/integration/hive/src/main/java - ${basedir}/streaming/src/main/java - ${basedir}/streaming/src/main/scala - ${basedir}/sdk/sdk/src/main/java - ${basedir}/index/bloom/src/main/java - ${basedir}/index/lucene/src/main/java - ${basedir}/index/secondary-index/src/main/scala - ${basedir}/index/secondary-index/src/main/java - - - - - - - - spark-2.4 - - true - - - 2.4 - 2.4.5 - 2.11 - 2.11.12 - - - - - org.codehaus.mojo - flatten-maven-plugin - 1.2.2 - - ${project.build.directory} - - - - - flatten - process-resources - - flatten - - - - - flatten.clean - clean - - clean - - - - - - org.eluder.coveralls - coveralls-maven-plugin - 4.3.0 - - opPwqWW41vYppv6KISea3u1TJvE1ugJ5Y - UTF-8 - - ${basedir}/target/carbondata-coverage-report/carbondata-coverage-report.xml - - - - ${basedir}/common/src/main/java - ${basedir}/core/src/main/java - ${basedir}/geo/src/main/java - ${basedir}/geo/src/main/scala - ${basedir}/processing/src/main/java - ${basedir}/hadoop/src/main/java - ${basedir}/integration/spark/src/main/scala - ${basedir}/integration/spark/src/main/spark2.4 - ${basedir}/integration/spark/src/main/common2.3and2.4 - ${basedir}/integration/spark/src/main/common2.4and3.1 - ${basedir}/integration/spark/src/main/java - ${basedir}/integration/hive/src/main/scala - ${basedir}/integration/hive/src/main/java - ${basedir}/streaming/src/main/java - ${basedir}/streaming/src/main/scala - ${basedir}/sdk/sdk/src/main/java - ${basedir}/index/bloom/src/main/java - ${basedir}/index/lucene/src/main/java - ${basedir}/index/secondary-index/src/main/scala - ${basedir}/index/secondary-index/src/main/java - - - - - - - - spark-3.1 - - 3.1 - 3.1.1 - 2.12 - 2.12.8 - 2.10.0 - - - - - org.codehaus.mojo - flatten-maven-plugin - 1.2.2 - - ${project.build.directory} - - - - - flatten - process-resources - - flatten - - - - - flatten.clean - clean - - clean - - - - - - org.eluder.coveralls - coveralls-maven-plugin - 4.3.0 - - opPwqWW41vYppv6KISea3u1TJvE1ugJ5Y - UTF-8 - - ${basedir}/target/carbondata-coverage-report/carbondata-coverage-report.xml - - - - ${basedir}/common/src/main/java - ${basedir}/core/src/main/java - ${basedir}/geo/src/main/java - ${basedir}/geo/src/main/scala - ${basedir}/processing/src/main/java - ${basedir}/hadoop/src/main/java - ${basedir}/integration/spark/src/main/scala - ${basedir}/integration/spark/src/main/spark3.1 - ${basedir}/integration/spark/src/main/common2.4and3.1 - ${basedir}/integration/spark/src/main/java - ${basedir}/integration/hive/src/main/scala - ${basedir}/integration/hive/src/main/java - ${basedir}/streaming/src/main/java - ${basedir}/streaming/src/main/scala - ${basedir}/sdk/sdk/src/main/java - ${basedir}/index/bloom/src/main/java - ${basedir}/index/lucene/src/main/java - ${basedir}/index/secondary-index/src/main/scala - ${basedir}/index/secondary-index/src/main/java - - - - - - include-all diff --git a/streaming/pom.xml b/streaming/pom.xml index e815d86ec98..b02d1b6a10d 100644 --- a/streaming/pom.xml +++ b/streaming/pom.xml @@ -132,123 +132,5 @@ true - - spark-2.3 - - 2.3 - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - src/main/spark3.1 - - - - - org.codehaus.mojo - build-helper-maven-plugin - 3.0.0 - - - add-source - generate-sources - - add-source - - - - src/main/spark2.x - - - - - - - - - - spark-2.4 - - true - - - 2.4 - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - src/main/spark3.1 - - - - - org.codehaus.mojo - build-helper-maven-plugin - 3.0.0 - - - add-source - generate-sources - - add-source - - - - src/main/spark2.x - - - - - - - - - - spark-3.1 - - 3.1.1 - 3.1 - - - - - org.apache.maven.plugins - maven-compiler-plugin - - - src/main/spark2.x - - - - - org.codehaus.mojo - build-helper-maven-plugin - 3.0.0 - - - add-source - generate-sources - - add-source - - - - src/main/spark3.1 - - - - - - - - diff --git a/streaming/src/main/scala/org/apache/carbondata/streaming/parser/RowStreamParserImp.scala b/streaming/src/main/scala/org/apache/carbondata/streaming/parser/RowStreamParserImp.scala index 145520edd70..6adb29cbe93 100644 --- a/streaming/src/main/scala/org/apache/carbondata/streaming/parser/RowStreamParserImp.scala +++ b/streaming/src/main/scala/org/apache/carbondata/streaming/parser/RowStreamParserImp.scala @@ -18,12 +18,11 @@ package org.apache.carbondata.streaming.parser import java.text.SimpleDateFormat -import java.util import org.apache.hadoop.conf.Configuration -import org.apache.spark.sql.Row +import org.apache.spark.sql.{Encoders, Row} import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.{ExpressionEncoder, RowEncoder} +import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder import org.apache.spark.sql.types.StructType import org.apache.carbondata.core.constants.CarbonCommonConstants @@ -43,14 +42,14 @@ class RowStreamParserImp extends CarbonStreamParser { var timeStampFormat: SimpleDateFormat = _ var dateFormat: SimpleDateFormat = _ - val complexDelimiters: util.ArrayList[String] = new util.ArrayList[String]() + val complexDelimiters: java.util.ArrayList[String] = new java.util.ArrayList[String]() var serializationNullFormat: String = _ override def initialize(configuration: Configuration, structType: StructType, isVarcharTypeMapping: Array[Boolean]): Unit = { this.configuration = configuration this.structType = structType - this.encoder = RowEncoder.apply(this.structType).resolveAndBind() + this.encoder = Encoders.row(this.structType).asInstanceOf[ExpressionEncoder[Row]] this.isVarcharTypeMapping = isVarcharTypeMapping this.timeStampFormat = new SimpleDateFormat( diff --git a/streaming/src/main/spark3.1/org/apache/carbondata/util/SparkStreamingUtil.scala b/streaming/src/main/scala/org/apache/carbondata/util/SparkStreamingUtil.scala similarity index 100% rename from streaming/src/main/spark3.1/org/apache/carbondata/util/SparkStreamingUtil.scala rename to streaming/src/main/scala/org/apache/carbondata/util/SparkStreamingUtil.scala diff --git a/streaming/src/main/spark2.x/org.apache.carbondata.util/SparkStreamingUtil.scala b/streaming/src/main/spark2.x/org.apache.carbondata.util/SparkStreamingUtil.scala deleted file mode 100644 index 399d2f42514..00000000000 --- a/streaming/src/main/spark2.x/org.apache.carbondata.util/SparkStreamingUtil.scala +++ /dev/null @@ -1,35 +0,0 @@ -/* - * 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.util - -import java.text.SimpleDateFormat - -import org.apache.spark.sql.Row -import org.apache.spark.sql.catalyst.InternalRow -import org.apache.spark.sql.catalyst.encoders.ExpressionEncoder - -object SparkStreamingUtil { - - def convertInternalRowToRow(expressionEncoder: ExpressionEncoder[Row]): InternalRow => Row = { - expressionEncoder.fromRow - } - - def checkInstant(value: Any, timeStampFormat: SimpleDateFormat): String = { - value.toString - } -}