Skip to content

Commit 533ead6

Browse files
committed
added a catalog API which can be extended by any external catalog
1 parent 41831ce commit 533ead6

File tree

41 files changed

+357
-244
lines changed

Some content is hidden

Large Commits have some content hidden by default. Use the searchbox below for content that may be hidden.

41 files changed

+357
-244
lines changed

core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java

+3
Original file line numberDiff line numberDiff line change
@@ -2876,4 +2876,7 @@ private CarbonCommonConstants() {
28762876

28772877
public static final String CARBON_ENABLE_SCHEMA_ENFORCEMENT_DEFAULT = "true";
28782878

2879+
@CarbonProperty
2880+
public static final String CARBON_CATALOG_IMPL = "carbon.catalog.impl";
2881+
28792882
}

core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java

-1
Original file line numberDiff line numberDiff line change
@@ -43,7 +43,6 @@
4343
import org.apache.carbondata.core.fileoperations.FileWriteOperation;
4444
import org.apache.carbondata.core.util.CarbonUtil;
4545
import org.apache.carbondata.core.util.ThreadLocalSessionInfo;
46-
4746
import org.apache.commons.io.FileUtils;
4847
import org.apache.commons.lang.StringUtils;
4948
import org.apache.hadoop.conf.Configuration;

core/src/main/java/org/apache/carbondata/core/metadata/converter/SchemaConverter.java

+1-2
Original file line numberDiff line numberDiff line change
@@ -61,8 +61,7 @@ org.apache.carbondata.format.TableSchema fromWrapperToExternalTableSchema(
6161
* @param tableName
6262
* @return
6363
*/
64-
org.apache.carbondata.format.TableInfo fromWrapperToExternalTableInfo(TableInfo wrapperTableInfo,
65-
String dbName, String tableName);
64+
org.apache.carbondata.format.TableInfo fromWrapperToExternalTableInfo(TableInfo wrapperTableInfo);
6665

6766
/**
6867
* @param externalSchemaEvolutionEntry

core/src/main/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImpl.java

+1-2
Original file line numberDiff line numberDiff line change
@@ -305,8 +305,7 @@ private org.apache.carbondata.format.BucketingInfo fromWrapperToExternalBucketin
305305
* convert from wrapper to external table info
306306
*/
307307
@Override
308-
public org.apache.carbondata.format.TableInfo fromWrapperToExternalTableInfo(
309-
TableInfo wrapperTableInfo, String dbName, String tableName) {
308+
public org.apache.carbondata.format.TableInfo fromWrapperToExternalTableInfo(TableInfo wrapperTableInfo) {
310309
org.apache.carbondata.format.TableSchema thriftFactTable =
311310
fromWrapperToExternalTableSchema(wrapperTableInfo.getFactTable());
312311
return new org.apache.carbondata.format.TableInfo(thriftFactTable, new ArrayList<>());

core/src/test/java/org/apache/carbondata/core/metadata/converter/ThriftWrapperSchemaConverterImplTest.java

+1-1
Original file line numberDiff line numberDiff line change
@@ -1425,7 +1425,7 @@ public List<org.apache.carbondata.format.SchemaEvolutionEntry> getSchema_evoluti
14251425
org.apache.carbondata.format.TableSchema thriftFactTable =
14261426
new org.apache.carbondata.format.TableSchema("tableId", thriftColumnSchemas, schemaEvol);
14271427
org.apache.carbondata.format.TableInfo actualResult = thriftWrapperSchemaConverter
1428-
.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName);
1428+
.fromWrapperToExternalTableInfo(wrapperTableInfo);
14291429
org.apache.carbondata.format.TableInfo expectedResult =
14301430
new org.apache.carbondata.format.TableInfo(thriftFactTable, new ArrayList<org.apache
14311431
.carbondata.format.TableSchema>());

core/src/test/java/org/apache/carbondata/core/metadata/schema/table/TableInfoTest.java

+2-4
Original file line numberDiff line numberDiff line change
@@ -93,8 +93,7 @@ private TableInfo getTableInfo(String databaseName, String tableName) {
9393
CarbonTable carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
9494
ThriftWrapperSchemaConverterImpl schemaConverter = new ThriftWrapperSchemaConverterImpl();
9595
org.apache.carbondata.format.TableInfo thriftTable = schemaConverter
96-
.fromWrapperToExternalTableInfo(carbonTable.getTableInfo(), carbonTable.getDatabaseName(),
97-
carbonTable.getTableName());
96+
.fromWrapperToExternalTableInfo(carbonTable.getTableInfo());
9897
assertTrue(null != thriftTable);
9998
}
10099

@@ -135,8 +134,7 @@ private TableInfo getTableInfo(String databaseName, String tableName) {
135134
CarbonTable carbonTable = CarbonTable.buildFromTableInfo(tableInfo);
136135
ThriftWrapperSchemaConverterImpl schemaConverter = new ThriftWrapperSchemaConverterImpl();
137136
org.apache.carbondata.format.TableInfo thriftTable = schemaConverter
138-
.fromWrapperToExternalTableInfo(carbonTable.getTableInfo(), carbonTable.getDatabaseName(),
139-
carbonTable.getTableName());
137+
.fromWrapperToExternalTableInfo(carbonTable.getTableInfo());
140138
assertTrue(null != thriftTable);
141139
}
142140

examples/spark/pom.xml

+3-3
Original file line numberDiff line numberDiff line change
@@ -200,9 +200,6 @@
200200
</profile>
201201
<profile>
202202
<id>spark-2.3</id>
203-
<activation>
204-
<activeByDefault>true</activeByDefault>
205-
</activation>
206203
<properties>
207204
<spark.binary.version>2.3</spark.binary.version>
208205
</properties>
@@ -215,6 +212,9 @@
215212
</profile>
216213
<profile>
217214
<id>spark-3.1</id>
215+
<activation>
216+
<activeByDefault>true</activeByDefault>
217+
</activation>
218218
<properties>
219219
<spark.binary.version>3.1</spark.binary.version>
220220
<dep.jackson.version>2.10.0</dep.jackson.version>

hadoop/src/main/java/org/apache/carbondata/hadoop/testutil/StoreCreator.java

+1-4
Original file line numberDiff line numberDiff line change
@@ -228,10 +228,7 @@ public CarbonTable createTable(
228228

229229
SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
230230
org.apache.carbondata.format.TableInfo thriftTableInfo =
231-
schemaConverter.fromWrapperToExternalTableInfo(
232-
tableInfo,
233-
tableInfo.getDatabaseName(),
234-
tableInfo.getFactTable().getTableName());
231+
schemaConverter.fromWrapperToExternalTableInfo(tableInfo);
235232
org.apache.carbondata.format.SchemaEvolutionEntry schemaEvolutionEntry =
236233
new org.apache.carbondata.format.SchemaEvolutionEntry(tableInfo.getLastUpdatedTime());
237234
thriftTableInfo.getFact_table().getSchema_evolution().getSchema_evolution_history()

index/secondary-index/pom.xml

+3-3
Original file line numberDiff line numberDiff line change
@@ -158,9 +158,6 @@
158158
</profile>
159159
<profile>
160160
<id>spark-2.3</id>
161-
<activation>
162-
<activeByDefault>true</activeByDefault>
163-
</activation>
164161
<properties>
165162
<spark.binary.version>2.3</spark.binary.version>
166163
</properties>
@@ -173,6 +170,9 @@
173170
</profile>
174171
<profile>
175172
<id>spark-3.1</id>
173+
<activation>
174+
<activeByDefault>true</activeByDefault>
175+
</activation>
176176
<properties>
177177
<spark.binary.version>3.1</spark.binary.version>
178178
</properties>

integration/flink/pom.xml

+3-3
Original file line numberDiff line numberDiff line change
@@ -220,9 +220,6 @@
220220
<profiles>
221221
<profile>
222222
<id>spark-2.3</id>
223-
<activation>
224-
<activeByDefault>true</activeByDefault>
225-
</activation>
226223
<properties>
227224
<spark.binary.version>2.3</spark.binary.version>
228225
</properties>
@@ -263,6 +260,9 @@
263260
</profile>
264261
<profile>
265262
<id>spark-3.1</id>
263+
<activation>
264+
<activeByDefault>true</activeByDefault>
265+
</activation>
266266
<properties>
267267
<spark.binary.version>3.1</spark.binary.version>
268268
</properties>

integration/hive/src/main/java/org/apache/carbondata/hive/util/HiveCarbonUtil.java

+1-2
Original file line numberDiff line numberDiff line change
@@ -255,8 +255,7 @@ private static void writeSchemaFile(TableInfo tableInfo) throws IOException {
255255
ThriftWriter thriftWriter = new ThriftWriter(schemaFilePath, false);
256256
thriftWriter.open(FileWriteOperation.OVERWRITE);
257257
thriftWriter.write(schemaConverter
258-
.fromWrapperToExternalTableInfo(tableInfo, tableInfo.getDatabaseName(),
259-
tableInfo.getFactTable().getTableName()));
258+
.fromWrapperToExternalTableInfo(tableInfo));
260259
thriftWriter.close();
261260
}
262261

integration/presto/src/test/scala/org/apache/carbondata/presto/util/CarbonDataStoreCreator.scala

+1-4
Original file line numberDiff line numberDiff line change
@@ -153,10 +153,7 @@ object CarbonDataStoreCreator {
153153
val schemaConverter: SchemaConverter =
154154
new ThriftWrapperSchemaConverterImpl()
155155
val thriftTableInfo: TableInfo =
156-
schemaConverter.fromWrapperToExternalTableInfo(
157-
tableInfo,
158-
tableInfo.getDatabaseName,
159-
tableInfo.getFactTable.getTableName)
156+
schemaConverter.fromWrapperToExternalTableInfo(tableInfo)
160157
val schemaEvolutionEntry: SchemaEvolutionEntry =
161158
new org.apache.carbondata.format.SchemaEvolutionEntry(
162159
tableInfo.getLastUpdatedTime)

integration/spark/pom.xml

+3-3
Original file line numberDiff line numberDiff line change
@@ -608,9 +608,6 @@
608608
</profile>
609609
<profile>
610610
<id>spark-2.3</id>
611-
<activation>
612-
<activeByDefault>true</activeByDefault>
613-
</activation>
614611
<properties>
615612
<spark.binary.version>2.3</spark.binary.version>
616613
</properties>
@@ -693,6 +690,9 @@
693690
</profile>
694691
<profile>
695692
<id>spark-3.1</id>
693+
<activation>
694+
<activeByDefault>true</activeByDefault>
695+
</activation>
696696
<properties>
697697
<spark.binary.version>3.1</spark.binary.version>
698698
</properties>

integration/spark/src/main/common2.4and3.1/org/apache/spark/sql/execution/CreateDataSourceTableCommand.scala

+4-4
Original file line numberDiff line numberDiff line change
@@ -22,8 +22,8 @@ import org.apache.spark.sql.{AnalysisException, Row, SparkSession}
2222
import org.apache.spark.sql.catalyst.catalog.{CatalogTable, CatalogTableType, CatalogUtils}
2323
import org.apache.spark.sql.execution.command.RunnableCommand
2424
import org.apache.spark.sql.util.CreateTableCommonUtil.getCatalogTable
25-
2625
import org.apache.carbondata.common.logging.LogServiceFactory
26+
import org.apache.spark.sql.catalog.{CarbonCatalogImpl, CatalogFactory}
2727

2828
case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boolean)
2929
extends RunnableCommand {
@@ -34,18 +34,18 @@ case class CreateDataSourceTableCommand(table: CatalogTable, ignoreIfExists: Boo
3434
assert(table.tableType != CatalogTableType.VIEW)
3535
assert(table.provider.isDefined)
3636
val sessionState = sparkSession.sessionState
37-
if (sessionState.catalog.tableExists(table.identifier)) {
37+
if (CatalogFactory.getCatalog.tableExists(table.identifier)(sparkSession)) {
3838
if (ignoreIfExists) {
3939
return Seq.empty[Row]
4040
} else {
4141
throw new AnalysisException(s"Table ${table.identifier.unquotedString} already exists.")
4242
}
4343
}
4444
val newTable: CatalogTable = getCatalogTable(sparkSession, sessionState, table, LOGGER)
45-
4645
// We will return Nil or throw exception at the beginning if the table already exists, so when
4746
// we reach here, the table should not exist and we should set `ignoreIfExists` to false.
48-
sessionState.catalog.createTable(newTable, ignoreIfExists = false, validateLocation = false)
47+
CatalogFactory.getCatalog.createTable(newTable, ignoreIfExists = false,
48+
validateLocation = false)(sparkSession)
4949
Seq.empty[Row]
5050
}
5151
}

integration/spark/src/main/scala/org/apache/carbondata/view/MVManagerInSpark.scala

+2-1
Original file line numberDiff line numberDiff line change
@@ -22,6 +22,7 @@ import java.util
2222
import scala.collection.JavaConverters._
2323

2424
import org.apache.spark.sql.{CarbonEnv, CarbonThreadUtil, SparkSession}
25+
import org.apache.spark.sql.catalog.CatalogFactory
2526

2627
import org.apache.carbondata.core.constants.CarbonCommonConstants
2728
import org.apache.carbondata.core.metadata.schema.table.CarbonTable
@@ -31,7 +32,7 @@ class MVManagerInSpark(session: SparkSession) extends MVManager {
3132
override def getDatabases: util.List[String] = {
3233
CarbonThreadUtil.threadSet(CarbonCommonConstants.CARBON_ENABLE_MV, "true")
3334
try {
34-
session.sessionState.catalog.listDatabases().asJava
35+
CatalogFactory.getCatalog.listDatabases()(session).asJava
3536
} finally {
3637
CarbonThreadUtil.threadUnset(CarbonCommonConstants.CARBON_ENABLE_MV)
3738
}

integration/spark/src/main/scala/org/apache/spark/sql/CarbonEnv.scala

+2-6
Original file line numberDiff line numberDiff line change
@@ -18,12 +18,9 @@
1818
package org.apache.spark.sql
1919

2020
import java.util.concurrent.ConcurrentHashMap
21-
2221
import org.apache.hadoop.fs.Path
23-
import org.apache.spark.internal.config.ConfigEntry
2422
import org.apache.spark.sql.catalyst.TableIdentifier
2523
import org.apache.spark.sql.catalyst.analysis.{NoSuchDatabaseException, NoSuchTableException}
26-
import org.apache.spark.sql.catalyst.catalog.SessionCatalog
2724
import org.apache.spark.sql.events.{MergeBloomIndexEventListener, MergeIndexEventListener}
2825
import org.apache.spark.sql.execution.command.CreateFunctionCommand
2926
import org.apache.spark.sql.execution.command.mutation.merge.udf.BlockPathsUDF
@@ -32,7 +29,6 @@ import org.apache.spark.sql.listeners._
3229
import org.apache.spark.sql.profiler.Profiler
3330
import org.apache.spark.sql.secondaryindex.events._
3431
import org.apache.spark.util.CarbonReflectionUtils
35-
3632
import org.apache.carbondata.common.logging.LogServiceFactory
3733
import org.apache.carbondata.core.constants.CarbonCommonConstants
3834
import org.apache.carbondata.core.datastore.impl.FileFactory
@@ -46,6 +42,7 @@ import org.apache.carbondata.processing.loading.events.LoadEvents.{LoadTablePost
4642
import org.apache.carbondata.spark.rdd.SparkReadSupport
4743
import org.apache.carbondata.spark.readsupport.SparkRowReadSupportImpl
4844
import org.apache.carbondata.view.{MVFunctions, TimeSeriesFunction}
45+
import org.apache.spark.sql.catalog.CatalogFactory
4946

5047
/**
5148
* Carbon Environment for unified context
@@ -342,8 +339,7 @@ object CarbonEnv {
342339
*/
343340
def getDatabaseLocation(dbName: String, sparkSession: SparkSession): String = {
344341
var databaseLocation =
345-
sparkSession.sessionState.catalog.asInstanceOf[SessionCatalog].getDatabaseMetadata(dbName)
346-
.locationUri.toString
342+
CatalogFactory.getCatalog.getDatabaseMetadata(dbName)(sparkSession).locationUri.toString
347343
// for default database and db ends with .db
348344
// check whether the carbon store and hive store is same or different.
349345
if ((!EnvHelper.isLegacy(sparkSession)) &&

integration/spark/src/main/scala/org/apache/spark/sql/execution/command/cache/CarbonShowCacheCommand.scala

+2-3
Original file line numberDiff line numberDiff line change
@@ -19,15 +19,13 @@ package org.apache.spark.sql.execution.command.cache
1919

2020
import scala.collection.JavaConverters._
2121
import scala.collection.mutable
22-
2322
import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
2423
import org.apache.spark.sql.AnalysisException
2524
import org.apache.spark.sql.catalyst.TableIdentifier
2625
import org.apache.spark.sql.catalyst.analysis.NoSuchTableException
2726
import org.apache.spark.sql.catalyst.expressions.AttributeReference
2827
import org.apache.spark.sql.execution.command.{Checker, MetadataCommand}
2928
import org.apache.spark.sql.types.StringType
30-
3129
import org.apache.carbondata.common.logging.LogServiceFactory
3230
import org.apache.carbondata.core.cache.CacheProvider
3331
import org.apache.carbondata.core.index.IndexStoreManager
@@ -40,6 +38,7 @@ import org.apache.carbondata.events.{OperationContext, OperationListenerBus, Sho
4038
import org.apache.carbondata.indexserver.IndexServer
4139
import org.apache.carbondata.spark.util.CarbonScalaUtil
4240
import org.apache.carbondata.spark.util.CommonUtil.bytesToDisplaySize
41+
import org.apache.spark.sql.catalog.CatalogFactory
4342

4443
case class CarbonShowCacheCommand(showExecutorCache: Boolean,
4544
tableIdentifier: Option[TableIdentifier],
@@ -158,7 +157,7 @@ case class CarbonShowCacheCommand(showExecutorCache: Boolean,
158157
}
159158
}
160159
var carbonTables = mutable.ArrayBuffer[CarbonTable]()
161-
sparkSession.sessionState.catalog.listTables(currentDatabase).foreach {
160+
CatalogFactory.getCatalog.listTables(currentDatabase)(sparkSession).foreach {
162161
tableIdent =>
163162
try {
164163
val carbonTable = CarbonEnv.getCarbonTable(tableIdent)(sparkSession)

integration/spark/src/main/scala/org/apache/spark/sql/execution/command/index/IndexRepairCommand.scala

+2-3
Original file line numberDiff line numberDiff line change
@@ -18,16 +18,15 @@
1818
package org.apache.spark.sql.execution.command.index
1919

2020
import scala.collection.JavaConverters._
21-
2221
import org.apache.spark.sql.{CarbonEnv, Row, SparkSession}
2322
import org.apache.spark.sql.catalyst.TableIdentifier
2423
import org.apache.spark.sql.execution.command.DataCommand
2524
import org.apache.spark.sql.hive.CarbonRelation
2625
import org.apache.spark.sql.index.CarbonIndexUtil
27-
2826
import org.apache.carbondata.common.logging.LogServiceFactory
2927
import org.apache.carbondata.core.metadata.index.IndexType
3028
import org.apache.carbondata.processing.loading.model.{CarbonDataLoadSchema, CarbonLoadModel}
29+
import org.apache.spark.sql.catalog.CatalogFactory
3130

3231
/**
3332
* Repair logic for reindex command on maintable/indextable
@@ -51,7 +50,7 @@ extends DataCommand {
5150
triggerRepair(tableIdentifier.table, databaseName, indexnameOp, segments, sparkSession)
5251
} else {
5352
// repairing si for all index tables in the mentioned database in the repair command
54-
sparkSession.sessionState.catalog.listTables(dbName).foreach {
53+
CatalogFactory.getCatalog.listTables(dbName)(sparkSession).foreach {
5554
tableIdent =>
5655
triggerRepair(tableIdent.table, dbName, indexnameOp, segments, sparkSession)
5756
}

integration/spark/src/main/scala/org/apache/spark/sql/execution/command/management/RefreshCarbonTableCommand.scala

+3-4
Original file line numberDiff line numberDiff line change
@@ -19,7 +19,6 @@ package org.apache.spark.sql.execution.command.management
1919

2020
import scala.collection.JavaConverters._
2121
import scala.collection.mutable
22-
2322
import org.apache.spark.sql._
2423
import org.apache.spark.sql.catalyst.CarbonParserUtil.initializeSpatialIndexInstance
2524
import org.apache.spark.sql.catalyst.TableIdentifier
@@ -28,7 +27,6 @@ import org.apache.spark.sql.catalyst.catalog.CatalogTypes.TablePartitionSpec
2827
import org.apache.spark.sql.execution.command.{AlterTableAddPartitionCommand, MetadataCommand}
2928
import org.apache.spark.sql.execution.command.table.CarbonCreateTableCommand
3029
import org.apache.spark.util.{AlterTableUtil, SparkUtil}
31-
3230
import org.apache.carbondata.common.logging.LogServiceFactory
3331
import org.apache.carbondata.core.constants.CarbonCommonConstants
3432
import org.apache.carbondata.core.datastore.impl.FileFactory
@@ -40,7 +38,8 @@ import org.apache.carbondata.core.metadata.schema.table.TableInfo
4038
import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema
4139
import org.apache.carbondata.core.statusmanager.{SegmentStatus, SegmentStatusManager}
4240
import org.apache.carbondata.core.util.path.CarbonTablePath
43-
import org.apache.carbondata.events.{withEvents, RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent}
41+
import org.apache.carbondata.events.{RefreshTablePostExecutionEvent, RefreshTablePreExecutionEvent, withEvents}
42+
import org.apache.spark.sql.catalog.CatalogFactory
4443

4544
/**
4645
* Command to register carbon table from existing carbon table data
@@ -96,7 +95,7 @@ case class RefreshCarbonTableCommand(
9695
initializeSpatialIndexInstance(tableProperties.get(SPATIAL_INDEX_CLASS),
9796
indexName, tableProperties.asScala)
9897
val tableIdentifier = new TableIdentifier(tableName, Some(tableInfo.getDatabaseName))
99-
if (sparkSession.sessionState.catalog.tableExists(tableIdentifier)) {
98+
if (CatalogFactory.getCatalog.tableExists(tableIdentifier)(sparkSession)) {
10099
// In direct upgrade scenario, if spatial table already exists then on refresh command,
101100
// update the property in metadata and fail table creation.
102101
LOGGER.info(s"Updating $SPATIAL_INDEX_INSTANCE table property on $tableName")

integration/spark/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableAddColumnCommand.scala

+1-1
Original file line numberDiff line numberDiff line change
@@ -89,7 +89,7 @@ private[sql] case class CarbonAlterTableAddColumnCommand(
8989
newCols = newCols.filter(x => !x.isComplexColumn)
9090
schemaEvolutionEntry.setAdded(newCols.toList.asJava)
9191
val thriftTable = schemaConverter
92-
.fromWrapperToExternalTableInfo(wrapperTableInfo, dbName, tableName)
92+
.fromWrapperToExternalTableInfo(wrapperTableInfo)
9393
// carbon columns based on schema order
9494
val carbonColumns = carbonTable.getCreateOrderColumn().asScala
9595
.collect { case carbonColumn if !carbonColumn.isInvisible => carbonColumn.getColumnSchema }

integration/spark/src/main/scala/org/apache/spark/sql/execution/command/schema/CarbonAlterTableRenameCommand.scala

+2-2
Original file line numberDiff line numberDiff line change
@@ -24,7 +24,6 @@ import org.apache.spark.sql.execution.command.{AlterTableRenameModel, MetadataCo
2424
import org.apache.spark.sql.hive.{CarbonRelation, CarbonSessionCatalogUtil, MockClassForAlterRevertTests}
2525
import org.apache.spark.sql.index.CarbonIndexUtil
2626
import org.apache.spark.util.AlterTableUtil
27-
2827
import org.apache.carbondata.common.exceptions.sql.MalformedCarbonCommandException
2928
import org.apache.carbondata.common.logging.LogServiceFactory
3029
import org.apache.carbondata.core.exception.ConcurrentOperationException
@@ -38,6 +37,7 @@ import org.apache.carbondata.core.statusmanager.SegmentStatusManager
3837
import org.apache.carbondata.events.{AlterTableRenamePostEvent, AlterTableRenamePreEvent, OperationContext, OperationListenerBus}
3938
import org.apache.carbondata.format.SchemaEvolutionEntry
4039
import org.apache.carbondata.view.MVManagerInSpark
40+
import org.apache.spark.sql.catalog.CatalogFactory
4141

4242
private[sql] case class CarbonAlterTableRenameCommand(
4343
alterTableRenameModel: AlterTableRenameModel)
@@ -58,7 +58,7 @@ private[sql] case class CarbonAlterTableRenameCommand(
5858
if (!oldDatabaseName.equalsIgnoreCase(newDatabaseName)) {
5959
throw new MalformedCarbonCommandException("Database name should be same for both tables")
6060
}
61-
val tableExists = sparkSession.catalog.tableExists(oldDatabaseName, newTableIdentifier.table)
61+
val tableExists = CatalogFactory.getCatalog.tableExists(newTableIdentifier)(sparkSession)
6262
if (tableExists) {
6363
throw new MalformedCarbonCommandException(s"Table with name $newTableIdentifier " +
6464
s"already exists")

0 commit comments

Comments
 (0)