You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by ch...@apache.org on 2018/05/18 04:23:26 UTC

carbondata git commit: [CARBONDATA-2327] [Presto] Fixed invalid schema name _system shows when executed show schemas in presto

Repository: carbondata
Updated Branches:
  refs/heads/master 8fe165668 -> 15ab6b066


[CARBONDATA-2327] [Presto] Fixed invalid schema name _system shows when executed show schemas in presto

[Presto] Fixed invalid schema name _system shows when executed show schemas in presto

This closes #2164


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

Branch: refs/heads/master
Commit: 15ab6b06660e14d5b8a5b22556262db1a9d47012
Parents: 8fe1656
Author: anubhav100 <an...@knoldus.in>
Authored: Thu Apr 12 11:46:37 2018 +0530
Committer: chenliang613 <ch...@huawei.com>
Committed: Fri May 18 12:22:50 2018 +0800

----------------------------------------------------------------------
 .../carbondata/presto/CarbondataMetadata.java   |  1 +
 .../presto/impl/CarbonTableReader.java          | 20 ++++++++++++----
 .../integrationtest/PrestoAllDataTypeTest.scala | 24 ++++++++++++++++++++
 3 files changed, 40 insertions(+), 5 deletions(-)
----------------------------------------------------------------------


http://git-wip-us.apache.org/repos/asf/carbondata/blob/15ab6b06/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
index cc8043e..718c628 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/CarbondataMetadata.java
@@ -71,6 +71,7 @@ public class CarbondataMetadata implements ConnectorMetadata {
     ImmutableList.Builder<SchemaTableName> builder = ImmutableList.builder();
     for (String schemaName : schemaNames) {
       for (String tableName : carbonTableReader.getTableNames(schemaName)) {
+        if(!tableName.equalsIgnoreCase(".DS_Store"))
         builder.add(new SchemaTableName(schemaName, tableName));
       }
     }

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15ab6b06/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
----------------------------------------------------------------------
diff --git a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
index e8e986a..ad5b7ee 100755
--- a/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
+++ b/integration/presto/src/main/java/org/apache/carbondata/presto/impl/CarbonTableReader.java
@@ -123,6 +123,10 @@ public class CarbonTableReader {
   private static final LogService LOGGER =
       LogServiceFactory.getLogService(CarbonTableReader.class.getName());
 
+  /**
+   * List Of Schemas
+   */
+ private  List<String> schemaNames = new ArrayList<>();
 
   @Inject public CarbonTableReader(CarbonTableConfig config) {
     this.config = requireNonNull(config, "CarbonTableConfig is null");
@@ -203,12 +207,18 @@ public class CarbonTableReader {
    */
   private List<String> updateSchemaList() {
     updateCarbonFile();
-
     if (carbonFileList != null) {
-      return Stream.of(carbonFileList.listFiles()).map(CarbonFile::getName).collect(Collectors.toList());
+      Stream.of(carbonFileList.listFiles()).forEach(this::getName);
+      return schemaNames;
     } else return ImmutableList.of();
   }
 
+  private void getName(CarbonFile carbonFile){
+  if(!carbonFile.getName().equalsIgnoreCase("_system") && !carbonFile.getName().equalsIgnoreCase(".ds_store")){
+    schemaNames.add(carbonFile.getName());
+  }
+  }
+
   /**
    * Get the names of the tables in the given schema.
    *
@@ -278,13 +288,13 @@ public class CarbonTableReader {
     }
 
     if (isKeyExists) {
-      CarbonTableCacheModel ctcm = carbonCache.get().get(schemaTableName);
-      if(ctcm != null && ctcm.carbonTable.getTableInfo() != null) {
+      CarbonTableCacheModel carbonTableCacheModel = carbonCache.get().get(schemaTableName);
+      if(carbonTableCacheModel != null && carbonTableCacheModel.carbonTable.getTableInfo() != null) {
         Long latestTime = FileFactory.getCarbonFile(
             CarbonTablePath.getSchemaFilePath(
                 carbonCache.get().get(schemaTableName).carbonTable.getTablePath())
         ).getLastModifiedTime();
-        Long oldTime = ctcm.carbonTable.getTableInfo().getLastUpdatedTime();
+        Long oldTime = carbonTableCacheModel.carbonTable.getTableInfo().getLastUpdatedTime();
         if (DateUtils.truncate(new Date(latestTime), Calendar.MINUTE)
             .after(DateUtils.truncate(new Date(oldTime), Calendar.MINUTE))) {
           removeTableFromCache(schemaTableName);

http://git-wip-us.apache.org/repos/asf/carbondata/blob/15ab6b06/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
----------------------------------------------------------------------
diff --git a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
index ce17682..2e7402a 100644
--- a/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
+++ b/integration/presto/src/test/scala/org/apache/carbondata/presto/integrationtest/PrestoAllDataTypeTest.scala
@@ -19,9 +19,12 @@ package org.apache.carbondata.presto.integrationtest
 
 import java.io.File
 
+import org.apache.hadoop.fs.permission.{FsAction, FsPermission}
 import org.scalatest.{BeforeAndAfterAll, FunSuiteLike}
 
 import org.apache.carbondata.common.logging.LogServiceFactory
+import org.apache.carbondata.core.datastore.impl.FileFactory
+import org.apache.carbondata.core.datastore.impl.FileFactory.FileType
 import org.apache.carbondata.presto.server.PrestoServer
 
 
@@ -68,6 +71,7 @@ class PrestoAllDataTypeTest extends FunSuiteLike with BeforeAndAfterAll {
       .createCarbonStore(storePath,
         s"$rootPath/integration/presto/src/test/resources/alldatatype.csv")
     logger.info(s"\nCarbon store is created at location: $storePath")
+    cleanUp
     PrestoServer.startServer(storePath)
   }
 
@@ -497,4 +501,24 @@ class PrestoAllDataTypeTest extends FunSuiteLike with BeforeAndAfterAll {
     assert(actualResult.equals(expectedResult))
   }
 
+
+  test("test the show schemas result"){
+   val actualResult = PrestoServer.executeQuery("SHOW SCHEMAS")
+    assert(actualResult.equals(List(Map("Schema" -> "information_schema"), Map("Schema" -> "testdb"))))
+  }
+  test("test the show tables"){
+  val actualResult = PrestoServer.executeQuery("SHOW TABLES")
+  assert(actualResult.equals(List(Map("Table" -> "testtable"))))
+ }
+
+  private def cleanUp(): Unit = {
+    FileFactory.deleteFile(s"$storePath/Fact", FileType.LOCAL)
+    FileFactory
+      .createDirectoryAndSetPermission(s"$storePath/_system",
+        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL))
+    FileFactory
+      .createDirectoryAndSetPermission(s"$storePath/.DS_Store",
+        new FsPermission(FsAction.ALL, FsAction.ALL, FsAction.ALL))
+    FileFactory.createNewFile(s"$storePath/testdb/.DS_STORE",FileType.LOCAL)
+  }
 }
\ No newline at end of file