You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ma...@apache.org on 2018/05/17 14:30:32 UTC

[19/50] [abbrv] carbondata git commit: [CARBONDATA-2442][CARBONDATA-2469] Fixed: multiple issues in sdk writer and external table

[CARBONDATA-2442][CARBONDATA-2469] Fixed: multiple issues in sdk writer and external table

problem1: when two sdk writer output with differnt schema is placed in
same folder for reading, output is not as expected. It has many null
output.

root cause: when multiple carbondata and indexx files is placed in same
folder. table schema is inferred by first file.
comparing table schema with all other index file schema validation is
not present

solution: compare table schema with all other index file schema, if
there is a mismatch throw exception

problem2: External Table must show its location instead of default store location
solution: For external tables, show the carbon table path instead of default store location
in describe formatted

This closes #2273


Project: http://git-wip-us.apache.org/repos/asf/carbondata/repo
Commit: http://git-wip-us.apache.org/repos/asf/carbondata/commit/1dfbcfcc
Tree: http://git-wip-us.apache.org/repos/asf/carbondata/tree/1dfbcfcc
Diff: http://git-wip-us.apache.org/repos/asf/carbondata/diff/1dfbcfcc

Branch: refs/heads/spark-2.3
Commit: 1dfbcfccca5302fd02c31d2c2386cafa75b1f10c
Parents: fe436c3
Author: ajantha-bhat <aj...@gmail.com>
Authored: Sat May 5 16:59:44 2018 +0530
Committer: kunal642 <ku...@gmail.com>
Committed: Thu May 10 23:02:43 2018 +0530

----------------------------------------------------------------------
 .../schema/table/column/ColumnSchema.java       |  30 ++++++
 .../LatestFilesReadCommittedScope.java          |   5 +
 .../apache/carbondata/core/util/CarbonUtil.java |  64 +++++++++---
 .../hadoop/api/CarbonTableInputFormat.java      |  43 ++++++++
 .../createTable/TestCreateExternalTable.scala   |   2 +
 .../TestNonTransactionalCarbonTable.scala       | 103 ++++++++++++++++++-
 .../table/CarbonDescribeFormattedCommand.scala  |   4 +-
 7 files changed, 233 insertions(+), 18 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/1dfbcfcc/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
index edede18..1f05f63 100644
--- a/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
+++ b/core/src/main/java/org/apache/carbondata/core/metadata/schema/table/column/ColumnSchema.java
@@ -343,6 +343,36 @@ public class ColumnSchema implements Serializable, Writable {
   }
 
   /**
+   * method to compare columnSchema,
+   * other parameters along with just column name and column data type
+   * @param obj
+   * @return
+   */
+  public boolean equalsWithStrictCheck(Object obj) {
+    if (!this.equals(obj)) {
+      return false;
+    }
+    ColumnSchema other = (ColumnSchema) obj;
+    if (!columnUniqueId.equals(other.columnUniqueId) ||
+        (isDimensionColumn != other.isDimensionColumn) ||
+        (scale != other.scale) ||
+        (precision != other.precision) ||
+        (isSortColumn != other.isSortColumn)) {
+      return false;
+    }
+    if (encodingList.size() != other.encodingList.size()) {
+      return false;
+    }
+    for (int i = 0; i < encodingList.size(); i++) {
+      if (encodingList.get(i).compareTo(other.encodingList.get(i)) != 0) {
+        return false;
+      }
+    }
+
+    return true;
+  }
+
+  /**
    * @return the dataType
    */
   public DataType getDataType() {

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1dfbcfcc/core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java
----------------------------------------------------------------------
diff --git a/core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java b/core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java
index 3f870b8..8abf537 100644
--- a/core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java
+++ b/core/src/main/java/org/apache/carbondata/core/readcommitter/LatestFilesReadCommittedScope.java
@@ -122,6 +122,11 @@ public class LatestFilesReadCommittedScope implements ReadCommittedScope {
   @Override public void takeCarbonIndexFileSnapShot() throws IOException {
     // Read the current file Path get the list of indexes from the path.
     CarbonFile file = FileFactory.getCarbonFile(carbonFilePath);
+    if (file == null) {
+      // For nonTransactional table, files can be removed at any point of time.
+      // So cannot assume files will be present
+      throw new IOException("No files are present in the table location :"+ carbonFilePath);
+    }
     Map<String, List<String>> indexFileStore = new HashMap<>();
     if (file.isDirectory()) {
       CarbonFile[] carbonIndexFiles = SegmentIndexFileStore.getCarbonIndexFiles(carbonFilePath);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1dfbcfcc/core/src/main/java/org/apache/carbondata/core/util/CarbonUtil.java
----------------------------------------------------------------------
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 b42167d..ac0a800 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
@@ -73,6 +73,7 @@ import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.metadata.schema.table.column.ParentColumnTableRelation;
 import org.apache.carbondata.core.mutate.UpdateVO;
 import org.apache.carbondata.core.reader.CarbonHeaderReader;
+import org.apache.carbondata.core.reader.CarbonIndexFileReader;
 import org.apache.carbondata.core.reader.ThriftReader;
 import org.apache.carbondata.core.reader.ThriftReader.TBaseCreator;
 import org.apache.carbondata.core.scan.model.ProjectionDimension;
@@ -2354,6 +2355,56 @@ public final class CarbonUtil {
     }
     CarbonHeaderReader carbonHeaderReader = new CarbonHeaderReader(fistFilePath);
     List<ColumnSchema> columnSchemaList = carbonHeaderReader.readSchema();
+    // only columnSchema is the valid entry, reset all dummy entries.
+    TableSchema tableSchema = getDummyTableSchema(tableName,columnSchemaList);
+
+    ThriftWrapperSchemaConverterImpl thriftWrapperSchemaConverter =
+        new ThriftWrapperSchemaConverterImpl();
+    org.apache.carbondata.format.TableSchema thriftFactTable =
+        thriftWrapperSchemaConverter.fromWrapperToExternalTableSchema(tableSchema);
+    org.apache.carbondata.format.TableInfo tableInfo =
+        new org.apache.carbondata.format.TableInfo(thriftFactTable,
+            new ArrayList<org.apache.carbondata.format.TableSchema>());
+
+    tableInfo.setDataMapSchemas(null);
+    return tableInfo;
+  }
+
+  /**
+   * This method will infer the schema file from a given index file path
+   * @param indexFilePath
+   * @param tableName
+   * @return
+   * @throws IOException
+   */
+  public static org.apache.carbondata.format.TableInfo inferSchemaFromIndexFile(
+      String indexFilePath, String tableName) throws IOException {
+    CarbonIndexFileReader indexFileReader = new CarbonIndexFileReader();
+    indexFileReader.openThriftReader(indexFilePath);
+    org.apache.carbondata.format.IndexHeader readIndexHeader = indexFileReader.readIndexHeader();
+    List<ColumnSchema> columnSchemaList = new ArrayList<ColumnSchema>();
+    List<org.apache.carbondata.format.ColumnSchema> table_columns =
+        readIndexHeader.getTable_columns();
+    for (int i = 0; i < table_columns.size(); i++) {
+      columnSchemaList.add(thriftColumnSchmeaToWrapperColumnSchema(table_columns.get(i)));
+    }
+    // only columnSchema is the valid entry, reset all dummy entries.
+    TableSchema tableSchema = getDummyTableSchema(tableName, columnSchemaList);
+
+    ThriftWrapperSchemaConverterImpl thriftWrapperSchemaConverter =
+        new ThriftWrapperSchemaConverterImpl();
+    org.apache.carbondata.format.TableSchema thriftFactTable =
+        thriftWrapperSchemaConverter.fromWrapperToExternalTableSchema(tableSchema);
+    org.apache.carbondata.format.TableInfo tableInfo =
+        new org.apache.carbondata.format.TableInfo(thriftFactTable,
+            new ArrayList<org.apache.carbondata.format.TableSchema>());
+
+    tableInfo.setDataMapSchemas(null);
+    return tableInfo;
+  }
+
+  private static TableSchema getDummyTableSchema(String tableName,
+      List<ColumnSchema> columnSchemaList) {
     TableSchema tableSchema = new TableSchema();
     tableSchema.setTableName(tableName);
     tableSchema.setBucketingInfo(null);
@@ -2361,8 +2412,6 @@ public final class CarbonUtil {
     tableSchema.setTableId(UUID.randomUUID().toString());
     tableSchema.setListOfColumns(columnSchemaList);
 
-    ThriftWrapperSchemaConverterImpl thriftWrapperSchemaConverter =
-        new ThriftWrapperSchemaConverterImpl();
     SchemaEvolutionEntry schemaEvolutionEntry = new SchemaEvolutionEntry();
     schemaEvolutionEntry.setTimeStamp(System.currentTimeMillis());
     SchemaEvolution schemaEvol = new SchemaEvolution();
@@ -2370,18 +2419,9 @@ public final class CarbonUtil {
     schEntryList.add(schemaEvolutionEntry);
     schemaEvol.setSchemaEvolutionEntryList(schEntryList);
     tableSchema.setSchemaEvalution(schemaEvol);
-
-    org.apache.carbondata.format.TableSchema thriftFactTable =
-        thriftWrapperSchemaConverter.fromWrapperToExternalTableSchema(tableSchema);
-    org.apache.carbondata.format.TableInfo tableInfo =
-        new org.apache.carbondata.format.TableInfo(thriftFactTable,
-            new ArrayList<org.apache.carbondata.format.TableSchema>());
-
-    tableInfo.setDataMapSchemas(null);
-    return tableInfo;
+    return tableSchema;
   }
 
-
   public static void dropDatabaseDirectory(String databasePath)
       throws IOException, InterruptedException {
     FileFactory.FileType fileType = FileFactory.getFileType(databasePath);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1dfbcfcc/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
----------------------------------------------------------------------
diff --git a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
index 990e9a7..a32e17a 100644
--- a/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
+++ b/hadoop/src/main/java/org/apache/carbondata/hadoop/api/CarbonTableInputFormat.java
@@ -36,11 +36,14 @@ import org.apache.carbondata.core.datastore.impl.FileFactory;
 import org.apache.carbondata.core.indexstore.ExtendedBlocklet;
 import org.apache.carbondata.core.indexstore.PartitionSpec;
 import org.apache.carbondata.core.metadata.AbsoluteTableIdentifier;
+import org.apache.carbondata.core.metadata.converter.SchemaConverter;
+import org.apache.carbondata.core.metadata.converter.ThriftWrapperSchemaConverterImpl;
 import org.apache.carbondata.core.metadata.schema.PartitionInfo;
 import org.apache.carbondata.core.metadata.schema.SchemaReader;
 import org.apache.carbondata.core.metadata.schema.partition.PartitionType;
 import org.apache.carbondata.core.metadata.schema.table.CarbonTable;
 import org.apache.carbondata.core.metadata.schema.table.TableInfo;
+import org.apache.carbondata.core.metadata.schema.table.column.ColumnSchema;
 import org.apache.carbondata.core.mutate.CarbonUpdateUtil;
 import org.apache.carbondata.core.mutate.SegmentUpdateDetails;
 import org.apache.carbondata.core.mutate.UpdateVO;
@@ -149,6 +152,35 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
     SegmentStatusManager segmentStatusManager = new SegmentStatusManager(identifier);
     SegmentStatusManager.ValidAndInvalidSegmentsInfo segments = segmentStatusManager
         .getValidAndInvalidSegments(loadMetadataDetails, this.readCommittedScope);
+
+    // For NonTransactional table, compare the schema of all index files with inferred schema.
+    // If there is a mismatch throw exception. As all files must be of same schema.
+    if (!carbonTable.getTableInfo().isTransactionalTable()) {
+      SchemaConverter schemaConverter = new ThriftWrapperSchemaConverterImpl();
+      for (Segment segment : segments.getValidSegments()) {
+        Map<String, String> indexFiles = segment.getCommittedIndexFile();
+        for (Map.Entry<String, String> indexFileEntry : indexFiles.entrySet()) {
+          Path indexFile = new Path(indexFileEntry.getKey());
+          org.apache.carbondata.format.TableInfo tableInfo = CarbonUtil.inferSchemaFromIndexFile(
+              indexFile.toString(), carbonTable.getTableName());
+          TableInfo wrapperTableInfo = schemaConverter.fromExternalToWrapperTableInfo(
+              tableInfo, identifier.getDatabaseName(),
+              identifier.getTableName(),
+              identifier.getTablePath());
+          List<ColumnSchema> indexFileColumnList =
+              wrapperTableInfo.getFactTable().getListOfColumns();
+          List<ColumnSchema> tableColumnList =
+              carbonTable.getTableInfo().getFactTable().getListOfColumns();
+          if (!compareColumnSchemaList(indexFileColumnList, tableColumnList)) {
+            LOG.error("Schema of " + indexFile.getName()
+                + " doesn't match with the table's schema");
+            throw new IOException("All the files doesn't have same schema. "
+                + "Unsupported operation on nonTransactional table. Check logs.");
+          }
+        }
+      }
+    }
+
     // to check whether only streaming segments access is enabled or not,
     // if access streaming segment is true then data will be read from streaming segments
     boolean accessStreamingSegments = getAccessStreamingSegments(job.getConfiguration());
@@ -262,6 +294,17 @@ public class CarbonTableInputFormat<T> extends CarbonInputFormat<T> {
     return splits;
   }
 
+  private boolean compareColumnSchemaList(List<ColumnSchema> indexFileColumnList,
+      List<ColumnSchema> tableColumnList) {
+    if (indexFileColumnList.size() != tableColumnList.size()) {
+      return false;
+    }
+    for (int i = 0; i < tableColumnList.size(); i++) {
+      return indexFileColumnList.get(i).equalsWithStrictCheck(tableColumnList.get(i));
+    }
+    return false;
+  }
+
   /**
    * Below method will be used to get the filter segments when query is fired on pre Aggregate
    * and main table in case of streaming.

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1dfbcfcc/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
index 67370eb..3b21d0a 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestCreateExternalTable.scala
@@ -53,6 +53,8 @@ class TestCreateExternalTable extends QueryTest with BeforeAndAfterAll {
        """.stripMargin)
     checkAnswer(sql("SELECT count(*) from source"), sql("SELECT count(*) from origin"))
 
+    checkExistence(sql("describe formatted source"), true, storeLocation+"/origin")
+
     val carbonTable = CarbonEnv.getCarbonTable(None, "source")(sqlContext.sparkSession)
     assert(carbonTable.isExternalTable)
     

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1dfbcfcc/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
----------------------------------------------------------------------
diff --git a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
index ca6ac3c..3c51efe 100644
--- a/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
+++ b/integration/spark-common-test/src/test/scala/org/apache/carbondata/spark/testsuite/createTable/TestNonTransactionalCarbonTable.scala
@@ -19,7 +19,6 @@ package org.apache.carbondata.spark.testsuite.createTable
 
 import java.sql.Timestamp
 import java.io.{File, FileFilter, IOException}
-import java.io.{File, FileFilter}
 import java.util
 
 import org.apache.commons.io.FileUtils
@@ -33,8 +32,7 @@ import org.apache.carbondata.core.constants.CarbonCommonConstants
 import org.apache.carbondata.core.datastore.filesystem.CarbonFile
 import org.apache.carbondata.core.datastore.impl.FileFactory
 import org.apache.carbondata.core.util.CarbonUtil
-import org.apache.carbondata.sdk.file.{CarbonWriter, CarbonWriterBuilder, Field, Schema}
-import org.apache.carbondata.sdk.file.{AvroCarbonWriter, CarbonWriter, Field, Schema}
+import org.apache.carbondata.sdk.file.AvroCarbonWriter
 import scala.collection.JavaConverters._
 import scala.collection.mutable
 
@@ -42,7 +40,7 @@ import org.apache.avro
 import org.apache.commons.lang.CharEncoding
 import tech.allegro.schema.json2avro.converter.JsonAvroConverter
 
-import org.apache.carbondata.core.metadata.datatype.{DataTypes, StructField}
+import org.apache.carbondata.core.metadata.datatype.DataTypes
 import org.apache.carbondata.sdk.file.{CarbonWriter, CarbonWriterBuilder, Field, Schema}
 
 
@@ -172,6 +170,35 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
     }
   }
 
+  // prepare sdk writer output with other schema
+  def buildTestDataOtherDataType(rows: Int, sortColumns: Array[String]): Any = {
+    val fields: Array[Field] = new Array[Field](3)
+    // same column name, but name as boolean type
+    fields(0) = new Field("name", DataTypes.BOOLEAN)
+    fields(1) = new Field("age", DataTypes.INT)
+    fields(2) = new Field("height", DataTypes.DOUBLE)
+
+    try {
+      val builder = CarbonWriter.builder()
+      val writer =
+        builder.withSchema(new Schema(fields)).outputPath(writerPath)
+          .isTransactionalTable(false)
+          .uniqueIdentifier(System.currentTimeMillis()).withBlockSize(2).sortBy(sortColumns)
+          .buildWriterForCSVInput()
+
+      var i = 0
+      while (i < rows) {
+        writer.write(Array[String]("true", String.valueOf(i), String.valueOf(i.toDouble / 2)))
+        i += 1
+      }
+      writer.close()
+    } catch {
+      case ex: Exception => throw new RuntimeException(ex)
+      case _ => None
+    }
+  }
+
+
   def cleanTestData() = {
     FileUtils.deleteDirectory(new File(writerPath))
   }
@@ -212,6 +239,8 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
 
     checkExistence(sql("describe formatted sdkOutputTable"), true, "age,name")
 
+    checkExistence(sql("describe formatted sdkOutputTable"), true, writerPath)
+
     sql("DROP TABLE sdkOutputTable")
     // drop table should not delete the files
     assert(new File(writerPath).exists())
@@ -719,6 +748,72 @@ class TestNonTransactionalCarbonTable extends QueryTest with BeforeAndAfterAll {
 
   }
 
+
+  test("Read sdk two writer output with same column name but different sort columns") {
+    FileUtils.deleteDirectory(new File(writerPath))
+    buildTestDataOtherDataType(3, Array[String]("name"))
+    assert(new File(writerPath).exists())
+
+    sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+    sql(
+      s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbondata' LOCATION
+         |'$writerPath' """.stripMargin)
+
+    checkAnswer(sql("select * from sdkOutputTable"), Seq(Row(true, 0, 0.0),
+      Row(true, 1, 0.5),
+      Row(true, 2, 1.0)))
+
+
+    buildTestDataOtherDataType(3, Array[String]("age"))
+    // put other sdk writer output to same path,
+    // which has same column names but different sort column
+    val exception =
+    intercept[IOException] {
+      sql("select * from sdkOutputTable").show(false)
+    }
+    assert(exception.getMessage()
+      .contains("All the files doesn't have same schema"))
+
+    sql("DROP TABLE sdkOutputTable")
+    // drop table should not delete the files
+    assert(new File(writerPath).exists())
+    cleanTestData()
+  }
+
+
+  test("Read sdk two writer output with same column name but different data type ") {
+    buildTestDataSingleFile()
+    assert(new File(writerPath).exists())
+
+    sql("DROP TABLE IF EXISTS sdkOutputTable")
+
+    sql(
+      s"""CREATE EXTERNAL TABLE sdkOutputTable STORED BY 'carbondata' LOCATION
+         |'$writerPath' """.stripMargin)
+
+    checkAnswer(sql("select * from sdkOutputTable"), Seq(Row("robot0", 0, 0.0),
+      Row("robot1", 1, 0.5),
+      Row("robot2", 2, 1.0)))
+
+    // put other sdk writer output to same path,
+    // which has same column names but different data type
+    buildTestDataOtherDataType(3, null)
+
+    val exception =
+      intercept[IOException] {
+        sql("select * from sdkOutputTable").show(false)
+      }
+    assert(exception.getMessage()
+      .contains("All the files doesn't have same schema"))
+
+
+    sql("DROP TABLE sdkOutputTable")
+    // drop table should not delete the files
+    assert(new File(writerPath).exists())
+    cleanTestData()
+  }
+
   private def WriteFilesWithAvroWriter(rows: Int, mySchema: String, json: String): Unit = {
     // conversion to GenericData.Record
     val nn = new avro.Schema.Parser().parse(mySchema)

http://git-wip-us.apache.org/repos/asf/carbondata/blob/1dfbcfcc/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
----------------------------------------------------------------------
diff --git a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
index 9b69373..375c413 100644
--- a/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
+++ b/integration/spark2/src/main/scala/org/apache/spark/sql/execution/command/table/CarbonDescribeFormattedCommand.scala
@@ -86,10 +86,10 @@ private[sql] case class CarbonDescribeFormattedCommand(
     results ++= Seq(("Database Name", relation.carbonTable.getDatabaseName, "")
     )
     results ++= Seq(("Table Name", relation.carbonTable.getTableName, ""))
-    if (carbonTable.isTransactionalTable) {
+    if (!carbonTable.isExternalTable) {
       results ++= Seq(("CARBON Store Path ", CarbonProperties.getStorePath, ""))
     } else {
-      // for NonTransactional table should show files path.
+      // for external table should show files path.
       results ++= Seq(("CARBON Store Path ", carbonTable.getTablePath, ""))
     }