Skip to content
New issue

Have a question about this project? Sign up for a free GitHub account to open an issue and contact its maintainers and the community.

By clicking “Sign up for GitHub”, you agree to our terms of service and privacy statement. We’ll occasionally send you account related emails.

Already on GitHub? Sign in to your account

[WIP] drop table issue #4188

Open
wants to merge 1 commit into
base: master
Choose a base branch
from
Open
Show file tree
Hide file tree
Changes from all commits
Commits
File filter

Filter by extension

Filter by extension

Conversations
Failed to load comments.
Loading
Jump to
Jump to file
Failed to load files.
Loading
Diff view
Diff view
Original file line number Diff line number Diff line change
Expand Up @@ -394,13 +394,8 @@ object CarbonEnv {
isExternal: Boolean,
isTransactionalTable: Boolean
)(sparkSession: SparkSession): String = {
var tmpPath = location.getOrElse(
CarbonEnv.newTablePath(databaseNameOp, tableName)(sparkSession))
if (!isExternal && isTransactionalTable && location.isEmpty &&
(FileFactory.getCarbonFile(tmpPath).exists() || EnvHelper.isLegacy(sparkSession))) {
tmpPath = tmpPath + "_" + tableId
}
val path = new Path(tmpPath)
val path = new Path(location.getOrElse(
CarbonEnv.newTablePath(databaseNameOp, tableName)(sparkSession)))
val fs = path.getFileSystem(sparkSession.sparkContext.hadoopConfiguration)
fs.makeQualified(path).toString
}
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -153,18 +153,6 @@ case class CarbonCreateTableCommand(
CarbonEnv.getInstance(sparkSession).carbonMetaStore
.removeTableFromMetadata(dbName, tableName)

// Delete the folders created by this call if the actual path is different
val actualPath = CarbonEnv
.getCarbonTable(TableIdentifier(tableName, Option(dbName)))(sparkSession)
.getTablePath

if (!actualPath.equalsIgnoreCase(tablePath)) {
LOGGER
.error(
"TableAlreadyExists with path : " + actualPath + " So, deleting " + tablePath)
FileFactory.deleteAllCarbonFilesOfDir(FileFactory.getCarbonFile(tablePath))
}

// No need to throw for create if not exists
if (ifNotExistsSet) {
LOGGER.error(e, e)
Expand Down
Original file line number Diff line number Diff line change
Expand Up @@ -17,6 +17,7 @@

package org.apache.carbondata.spark.testsuite.createTable

import java.io.File
import java.util.concurrent.{Callable, Executors, ExecutorService, Future, TimeUnit}

import org.apache.spark.sql.AnalysisException
Expand Down Expand Up @@ -44,6 +45,22 @@ class TestCreateTableIfNotExists extends QueryTest with BeforeAndAfterAll {
}
}

test("test create table with consistent table location") {
val dbName = "testdb"
val tblName = "testtbl"
sql(s"drop database if exists $dbName cascade")
val dblocation = warehouse + File.separator + dbName
sql(s"create database $dbName location '$dblocation'")
sql(s"use $dbName")

// here, we make sure the carbontablepath contains uuid
val tbllocation = dblocation + File.separator + tblName
new File(tbllocation).mkdir()
assert(new File(tbllocation).exists())

sql(s"create table $tblName(a int, b string) stored as carbondata")
}

test("test create table if not exist concurrently") {

val executorService: ExecutorService = Executors.newFixedThreadPool(10)
Expand Down