You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@carbondata.apache.org by aj...@apache.org on 2020/04/27 16:15:04 UTC

[carbondata] branch master updated: [CARBONDATA-3777] Add HDFSLocalCarbonFile implementation to Use FileSystem's LocalFileSystem in cluster mode

This is an automated email from the ASF dual-hosted git repository.

ajantha pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/carbondata.git


The following commit(s) were added to refs/heads/master by this push:
     new cc674bf  [CARBONDATA-3777] Add HDFSLocalCarbonFile implementation to Use FileSystem's LocalFileSystem in cluster mode
cc674bf is described below

commit cc674bf0cd9e651916a19e79f045d51498113fda
Author: akashrn5 <ak...@gmail.com>
AuthorDate: Tue Apr 21 14:23:41 2020 +0530

    [CARBONDATA-3777] Add HDFSLocalCarbonFile implementation to Use FileSystem's LocalFileSystem in cluster mode
    
    Why is this PR needed?
    Currently LocalFile file implementation is JAVA's file implementation, which will give problem if we want to load the local file in cluster for instance.
    
    What changes were proposed in this PR?
    Implement a new class HDFSLocalCarbonFile, which extends HDFSCarbonFIle and when a file with local file scheme "file://" is given and trying to load in cluster, it takes the file as HDFSLocalCarbonFile and go ahead instead of failing which is current behaviour.
    
    Does this PR introduce any user interface change?
    Yes. (Doc update is not needed)
    
    Is any new testcase added?
    No(Existing HDFSCarbonFile tests will take care)
    
    This closes #3721
---
 .../core/constants/CarbonCommonConstants.java          |  2 ++
 .../core/datastore/impl/DefaultFileTypeProvider.java   |  1 +
 .../carbondata/core/datastore/impl/FileFactory.java    | 18 ++++++++++++++++--
 3 files changed, 19 insertions(+), 2 deletions(-)

diff --git a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
index da12ca7..732d8d1 100644
--- a/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
+++ b/core/src/main/java/org/apache/carbondata/core/constants/CarbonCommonConstants.java
@@ -1632,6 +1632,8 @@ public final class CarbonCommonConstants {
 
   public static final String S3A_PREFIX = "s3a://";
 
+  public static final String LOCAL_FS_URI = "file:///";
+
   /**
    * Access Key for s3n
    */
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java
index ae51f00..6ce9020 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/DefaultFileTypeProvider.java
@@ -103,6 +103,7 @@ public class DefaultFileTypeProvider implements FileTypeInterface {
       case LOCAL:
         return new LocalCarbonFile(FileFactory.getUpdatedFilePath(path));
       case HDFS:
+      case HDFS_LOCAL:
         return new HDFSCarbonFile(path, conf);
       case S3:
         return new S3CarbonFile(path, conf);
diff --git a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
index d2dc7b3..2fac335 100644
--- a/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
+++ b/core/src/main/java/org/apache/carbondata/core/datastore/impl/FileFactory.java
@@ -101,6 +101,7 @@ public final class FileFactory {
       case ALLUXIO:
       case VIEWFS:
       case S3:
+      case HDFS_LOCAL:
         return new DFSFileReaderImpl(configuration);
       default:
         return new FileReaderImpl();
@@ -142,6 +143,10 @@ public final class FileFactory {
         .startsWith(CarbonCommonConstants.S3A_PREFIX) || lowerCase
         .startsWith(CarbonCommonConstants.S3_PREFIX)) {
       return FileType.S3;
+    } else if (lowerCase.startsWith(CarbonCommonConstants.LOCAL_FILE_PREFIX) && !configuration
+        .get(CarbonCommonConstants.FS_DEFAULT_FS)
+        .equalsIgnoreCase(CarbonCommonConstants.LOCAL_FS_URI)) {
+      return FileType.HDFS_LOCAL;
     }
     return null;
   }
@@ -157,6 +162,10 @@ public final class FileFactory {
         .startsWith(CarbonCommonConstants.S3A_PREFIX) || path
         .startsWith(CarbonCommonConstants.S3_PREFIX)) {
       return FileType.S3;
+    } else if (path.startsWith(CarbonCommonConstants.LOCAL_FILE_PREFIX) && !configuration
+        .get(CarbonCommonConstants.FS_DEFAULT_FS)
+        .equalsIgnoreCase(CarbonCommonConstants.LOCAL_FS_URI)) {
+      return FileType.HDFS_LOCAL;
     }
     return null;
   }
@@ -425,7 +434,7 @@ public final class FileFactory {
   }
 
   public enum FileType {
-    LOCAL, HDFS, ALLUXIO, VIEWFS, S3, CUSTOM
+    LOCAL, HDFS, ALLUXIO, VIEWFS, S3, CUSTOM, HDFS_LOCAL
   }
 
   /**
@@ -447,6 +456,7 @@ public final class FileFactory {
       case VIEWFS:
       case S3:
       case CUSTOM:
+      case HDFS_LOCAL:
       default:
         return filePath;
     }
@@ -466,6 +476,7 @@ public final class FileFactory {
       case VIEWFS:
       case S3:
       case CUSTOM:
+      case HDFS_LOCAL:
         return filePath;
       case ALLUXIO:
         return StringUtils.startsWith(filePath, "alluxio") ? filePath : "alluxio:///" + filePath;
@@ -505,6 +516,7 @@ public final class FileFactory {
       case VIEWFS:
       case S3:
       case CUSTOM:
+      case HDFS_LOCAL:
         Path path = new Path(filePath);
         FileSystem fs = path.getFileSystem(getConfiguration());
         return fs.getContentSummary(path).getLength();
@@ -546,6 +558,7 @@ public final class FileFactory {
       case ALLUXIO:
       case VIEWFS:
       case CUSTOM:
+      case HDFS_LOCAL:
         try {
           Path path = new Path(directoryPath);
           FileSystem fs = path.getFileSystem(getConfiguration());
@@ -584,7 +597,8 @@ public final class FileFactory {
         .startsWith(CarbonCommonConstants.VIEWFSURL_PREFIX) || lowerPath
         .startsWith(CarbonCommonConstants.S3N_PREFIX) || lowerPath
         .startsWith(CarbonCommonConstants.S3A_PREFIX) || lowerPath
-        .startsWith(CarbonCommonConstants.S3_PREFIX)) {
+        .startsWith(CarbonCommonConstants.S3_PREFIX) || lowerPath
+        .startsWith(CarbonCommonConstants.LOCAL_FILE_PREFIX)) {
       return path;
     } else if (defaultFs != null) {
       return defaultFs + CarbonCommonConstants.FILE_SEPARATOR + path;