You are viewing a plain text version of this content. The canonical link for it is here.
Posted to gitbox@hive.apache.org by GitBox <gi...@apache.org> on 2021/09/28 14:42:24 UTC

[GitHub] [hive] kgyrtkirk opened a new pull request #2680: HIVE-25569: Enable table definition over a single file(SFS)

kgyrtkirk opened a new pull request #2680:
URL: https://github.com/apache/hive/pull/2680


   Change-Id: I6e8afa3463951c5b4e032df390df06a0d634fde7
   
   <!--
   Thanks for sending a pull request!  Here are some tips for you:
     1. If this is your first time, please read our contributor guidelines: https://cwiki.apache.org/confluence/display/Hive/HowToContribute
     2. Ensure that you have created an issue on the Hive project JIRA: https://issues.apache.org/jira/projects/HIVE/summary
     3. Ensure you have added or run the appropriate tests for your PR: 
     4. If the PR is unfinished, add '[WIP]' in your PR title, e.g., '[WIP]HIVE-XXXXX:  Your PR title ...'.
     5. Be sure to keep the PR description updated to reflect all changes.
     6. Please write your PR title to summarize what this PR proposes.
     7. If possible, provide a concise example to reproduce the issue for a faster review.
   
   -->
   
   ### What changes were proposed in this pull request?
   <!--
   Please clarify what changes you are proposing. The purpose of this section is to outline the changes and how this PR fixes the issue. 
   If possible, please consider writing useful notes for better and faster reviews in your PR. See the examples below.
     1. If you refactor some codes with changing classes, showing the class hierarchy will help reviewers.
     2. If you fix some SQL features, you can provide some references of other DBMSes.
     3. If there is design documentation, please add the link.
     4. If there is a discussion in the mailing list, please add the link.
   -->
   
   
   ### Why are the changes needed?
   <!--
   Please clarify why the changes are needed. For instance,
     1. If you propose a new API, clarify the use case for a new API.
     2. If you fix a bug, you can clarify why it is a bug.
   -->
   
   
   ### Does this PR introduce _any_ user-facing change?
   <!--
   Note that it means *any* user-facing change including all aspects such as the documentation fix.
   If yes, please clarify the previous behavior and the change this PR proposes - provide the console output, description, screenshot and/or a reproducable example to show the behavior difference if possible.
   If possible, please also clarify if this is a user-facing change compared to the released Hive versions or within the unreleased branches such as master.
   If no, write 'No'.
   -->
   
   
   ### How was this patch tested?
   <!--
   If tests were added, say they were added here. Please make sure to add some test cases that check the changes thoroughly including negative and positive cases if possible.
   If it was tested in a way different from regular unit tests, please clarify how you tested step by step, ideally copy and paste-able, so that other reviewers can test and check, and descendants can verify in the future.
   If tests were not added, please describe why they were not added and/or why it was difficult to add.
   -->
   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] kgyrtkirk merged pull request #2680: HIVE-25569: Enable table definition over a single file(SFS)

Posted by GitBox <gi...@apache.org>.
kgyrtkirk merged pull request #2680:
URL: https://github.com/apache/hive/pull/2680


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] kgyrtkirk commented on a change in pull request #2680: HIVE-25569: Enable table definition over a single file(SFS)

Posted by GitBox <gi...@apache.org>.
kgyrtkirk commented on a change in pull request #2680:
URL: https://github.com/apache/hive/pull/2680#discussion_r734301645



##########
File path: ql/src/java/org/apache/hadoop/hive/ql/io/SingleFileSystem.java
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.io;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Implements an abstraction layer to show files in a single directory.
+ *
+ * Suppose the filesystem has a directory in which there are multiple files:
+ * file://somedir/f1.txt
+ * file://somedir/f2.txt
+ *
+ * In case of Hive the content of a directory may be inside a table.
+ * To give a way to show a single file as a single file in a directory it could be specified:
+ *
+ * sfs+file://somedir/f1.txt/#SINGLEFILE#
+ *
+ * This will be a directory containing only the f1.txt and nothing else.
+ *
+ */
+public abstract class SingleFileSystem extends FileSystem {
+
+  public static class HDFS extends SingleFileSystem {
+  }
+
+  public static class S3A extends SingleFileSystem {
+  }
+
+  public static class ABFS extends SingleFileSystem {
+  }
+
+  public static class ABFSS extends SingleFileSystem {
+  }
+
+  public static class ADL extends SingleFileSystem {
+  }
+
+  public static class GS extends SingleFileSystem {
+  }
+
+  public static class O3FS extends SingleFileSystem {
+  }
+
+  public static class PFILE extends SingleFileSystem {
+  }
+
+  public static class FILE extends SingleFileSystem {
+  }
+
+  private static final String SINGLEFILE = "#SINGLEFILE#";
+
+  private URI uri;
+  private Configuration conf;
+  private Path workDir;
+
+  public String getScheme() {
+    return "sfs+" + getClass().getSimpleName().toLowerCase();
+  }
+
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    this.uri = uri;
+    this.conf = conf;
+  }
+
+  @Override
+  public URI getUri() {
+    return uri;
+  }
+
+  @Override
+  public FSDataInputStream open(Path upperPath, int bufferSize) throws IOException {
+    SfsInfo info = new SfsInfo(upperPath);
+    switch (info.type) {
+    case LEAF_FILE:
+      return info.lowerTargetPath.getFileSystem(conf).open(info.lowerTargetPath, bufferSize);
+    default:
+      throw unsupported("open:" + upperPath);
+    }
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path upperPath) throws IOException {
+    SfsInfo info = new SfsInfo(upperPath);
+    switch (info.type) {
+    case LEAF_FILE:
+      return info.getTargetFileStatus();
+    case DIR_MODE:
+    case SINGLEFILE_DIR:
+      return makeDirFileStatus(upperPath, info.lowerTargetPath);
+    default:
+      throw unsupported("fileStatus:" + upperPath);
+    }
+  }
+
+  enum SfsInodeType {
+    LEAF_FILE, SINGLEFILE_DIR, DIR_MODE, NONEXISTENT,
+  }
+
+  class SfsInfo {
+
+    private String[] parts;

Review comment:
       move to local

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/io/SingleFileSystem.java
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.io;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Implements an abstraction layer to show files in a single directory.
+ *
+ * Suppose the filesystem has a directory in which there are multiple files:
+ * file://somedir/f1.txt
+ * file://somedir/f2.txt
+ *
+ * In case of Hive the content of a directory may be inside a table.
+ * To give a way to show a single file as a single file in a directory it could be specified:
+ *
+ * sfs+file://somedir/f1.txt/#SINGLEFILE#
+ *
+ * This will be a directory containing only the f1.txt and nothing else.
+ *
+ */
+public abstract class SingleFileSystem extends FileSystem {
+
+  public static class HDFS extends SingleFileSystem {
+  }
+
+  public static class S3A extends SingleFileSystem {
+  }
+
+  public static class ABFS extends SingleFileSystem {
+  }
+
+  public static class ABFSS extends SingleFileSystem {
+  }
+
+  public static class ADL extends SingleFileSystem {
+  }
+
+  public static class GS extends SingleFileSystem {
+  }
+
+  public static class O3FS extends SingleFileSystem {
+  }
+
+  public static class PFILE extends SingleFileSystem {
+  }
+
+  public static class FILE extends SingleFileSystem {
+  }
+
+  private static final String SINGLEFILE = "#SINGLEFILE#";
+
+  private URI uri;
+  private Configuration conf;
+  private Path workDir;
+
+  public String getScheme() {
+    return "sfs+" + getClass().getSimpleName().toLowerCase();
+  }
+
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    this.uri = uri;
+    this.conf = conf;
+  }
+
+  @Override
+  public URI getUri() {
+    return uri;
+  }
+
+  @Override
+  public FSDataInputStream open(Path upperPath, int bufferSize) throws IOException {
+    SfsInfo info = new SfsInfo(upperPath);
+    switch (info.type) {
+    case LEAF_FILE:
+      return info.lowerTargetPath.getFileSystem(conf).open(info.lowerTargetPath, bufferSize);
+    default:
+      throw unsupported("open:" + upperPath);
+    }
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path upperPath) throws IOException {
+    SfsInfo info = new SfsInfo(upperPath);
+    switch (info.type) {
+    case LEAF_FILE:
+      return info.getTargetFileStatus();
+    case DIR_MODE:
+    case SINGLEFILE_DIR:
+      return makeDirFileStatus(upperPath, info.lowerTargetPath);
+    default:
+      throw unsupported("fileStatus:" + upperPath);
+    }
+  }
+
+  enum SfsInodeType {
+    LEAF_FILE, SINGLEFILE_DIR, DIR_MODE, NONEXISTENT,
+  }
+
+  class SfsInfo {
+
+    private String[] parts;
+    private URI uri;

Review comment:
       add final

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/io/SingleFileSystem.java
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.io;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Implements an abstraction layer to show files in a single directory.
+ *
+ * Suppose the filesystem has a directory in which there are multiple files:
+ * file://somedir/f1.txt
+ * file://somedir/f2.txt
+ *
+ * In case of Hive the content of a directory may be inside a table.
+ * To give a way to show a single file as a single file in a directory it could be specified:
+ *
+ * sfs+file://somedir/f1.txt/#SINGLEFILE#
+ *
+ * This will be a directory containing only the f1.txt and nothing else.
+ *
+ */
+public abstract class SingleFileSystem extends FileSystem {
+
+  public static class HDFS extends SingleFileSystem {
+  }
+
+  public static class S3A extends SingleFileSystem {
+  }
+
+  public static class ABFS extends SingleFileSystem {
+  }
+
+  public static class ABFSS extends SingleFileSystem {
+  }
+
+  public static class ADL extends SingleFileSystem {
+  }
+
+  public static class GS extends SingleFileSystem {
+  }
+
+  public static class O3FS extends SingleFileSystem {
+  }
+
+  public static class PFILE extends SingleFileSystem {
+  }
+
+  public static class FILE extends SingleFileSystem {
+  }
+
+  private static final String SINGLEFILE = "#SINGLEFILE#";
+
+  private URI uri;
+  private Configuration conf;
+  private Path workDir;
+
+  public String getScheme() {
+    return "sfs+" + getClass().getSimpleName().toLowerCase();
+  }
+
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    this.uri = uri;
+    this.conf = conf;
+  }
+
+  @Override
+  public URI getUri() {
+    return uri;
+  }
+
+  @Override
+  public FSDataInputStream open(Path upperPath, int bufferSize) throws IOException {
+    SfsInfo info = new SfsInfo(upperPath);
+    switch (info.type) {
+    case LEAF_FILE:
+      return info.lowerTargetPath.getFileSystem(conf).open(info.lowerTargetPath, bufferSize);
+    default:
+      throw unsupported("open:" + upperPath);
+    }
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path upperPath) throws IOException {
+    SfsInfo info = new SfsInfo(upperPath);
+    switch (info.type) {
+    case LEAF_FILE:
+      return info.getTargetFileStatus();
+    case DIR_MODE:
+    case SINGLEFILE_DIR:
+      return makeDirFileStatus(upperPath, info.lowerTargetPath);
+    default:
+      throw unsupported("fileStatus:" + upperPath);
+    }
+  }
+
+  enum SfsInodeType {
+    LEAF_FILE, SINGLEFILE_DIR, DIR_MODE, NONEXISTENT,

Review comment:
       apidoc

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/io/SingleFileSystem.java
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.io;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Implements an abstraction layer to show files in a single directory.
+ *
+ * Suppose the filesystem has a directory in which there are multiple files:
+ * file://somedir/f1.txt
+ * file://somedir/f2.txt
+ *
+ * In case of Hive the content of a directory may be inside a table.
+ * To give a way to show a single file as a single file in a directory it could be specified:
+ *
+ * sfs+file://somedir/f1.txt/#SINGLEFILE#
+ *
+ * This will be a directory containing only the f1.txt and nothing else.
+ *
+ */
+public abstract class SingleFileSystem extends FileSystem {
+
+  public static class HDFS extends SingleFileSystem {
+  }
+
+  public static class S3A extends SingleFileSystem {
+  }
+
+  public static class ABFS extends SingleFileSystem {
+  }
+
+  public static class ABFSS extends SingleFileSystem {
+  }
+
+  public static class ADL extends SingleFileSystem {
+  }
+
+  public static class GS extends SingleFileSystem {
+  }
+
+  public static class O3FS extends SingleFileSystem {
+  }
+
+  public static class PFILE extends SingleFileSystem {
+  }
+
+  public static class FILE extends SingleFileSystem {
+  }
+
+  private static final String SINGLEFILE = "#SINGLEFILE#";
+
+  private URI uri;
+  private Configuration conf;
+  private Path workDir;
+
+  public String getScheme() {
+    return "sfs+" + getClass().getSimpleName().toLowerCase();
+  }
+
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    this.uri = uri;
+    this.conf = conf;
+  }
+
+  @Override
+  public URI getUri() {
+    return uri;
+  }
+
+  @Override
+  public FSDataInputStream open(Path upperPath, int bufferSize) throws IOException {
+    SfsInfo info = new SfsInfo(upperPath);
+    switch (info.type) {
+    case LEAF_FILE:
+      return info.lowerTargetPath.getFileSystem(conf).open(info.lowerTargetPath, bufferSize);
+    default:
+      throw unsupported("open:" + upperPath);
+    }
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path upperPath) throws IOException {
+    SfsInfo info = new SfsInfo(upperPath);
+    switch (info.type) {
+    case LEAF_FILE:
+      return info.getTargetFileStatus();
+    case DIR_MODE:
+    case SINGLEFILE_DIR:
+      return makeDirFileStatus(upperPath, info.lowerTargetPath);
+    default:
+      throw unsupported("fileStatus:" + upperPath);
+    }
+  }
+
+  enum SfsInodeType {
+    LEAF_FILE, SINGLEFILE_DIR, DIR_MODE, NONEXISTENT,
+  }
+
+  class SfsInfo {
+
+    private String[] parts;
+    private URI uri;
+    private SfsInodeType type;
+    private Path lowerTargetPath;
+    private Path upperTargetPath;
+
+    public SfsInfo(Path upperPath) {
+      uri = upperPath.toUri();
+      parts = uri.getPath().split(Path.SEPARATOR);
+      type = SfsInodeType.DIR_MODE;
+      lowerTargetPath = upperPath;
+
+      int n = parts.length;
+      if (n >= 1 && parts[n - 1].equals(SINGLEFILE)) {
+        type = SfsInodeType.SINGLEFILE_DIR;
+        lowerTargetPath = upperPath.getParent();
+        upperTargetPath = new Path(uri.getScheme(), uri.getAuthority(), uri.getPath() + "/" + parts[n - 2]);
+      }
+      if (n >= 2 && parts[n - 2].equals(SINGLEFILE)) {
+        if (n >= 3 && !parts[n - 3].equals(parts[n - 1])) {
+          type = SfsInodeType.NONEXISTENT;
+        } else {
+          type = SfsInodeType.LEAF_FILE;
+          lowerTargetPath = upperPath.getParent().getParent();
+          upperTargetPath = upperPath;
+        }
+      }
+      if (lowerTargetPath != null) {
+        URI u = lowerTargetPath.toUri();
+        lowerTargetPath = new Path(removeSfsScheme(u.getScheme()), u.getAuthority(), u.getPath());
+      }
+    }
+
+    private String removeSfsScheme(String scheme) {
+      if (scheme.startsWith("sfs+")) {
+        return scheme.substring(4);
+      }
+      if (scheme.equals("sfs")) {
+        return null;
+      }
+      throw new RuntimeException("Unexpected scheme: " + scheme);
+    }
+
+    public FileStatus getTargetFileStatus() throws IOException {
+      return makeFileStatus(upperTargetPath, lowerTargetPath);
+    }
+
+    public FileStatus[] listStatus(Path upperPath, Path lowerPath) throws IOException {
+      FileSystem fs = lowerPath.getFileSystem(conf);
+      FileStatus status = fs.getFileStatus(lowerPath);
+      List<FileStatus> ret = new ArrayList<>();
+      if (status.isDirectory()) {
+        FileStatus[] statusList = fs.listStatus(lowerTargetPath);
+        for (FileStatus fileStatus : statusList) {
+          ret.add(makeDirFileStatus(fileStatus));
+        }
+      } else {
+        FileStatus dirStat = makeDirFileStatus(new Path(upperPath, SINGLEFILE), lowerPath);
+        ret.add(dirStat);
+      }
+      return ret.toArray(new FileStatus[0]);
+    }
+  }
+
+  @Override
+  public FileStatus[] listStatus(Path upperPath) throws FileNotFoundException, IOException {
+    SfsInfo info = new SfsInfo(upperPath);
+    switch (info.type) {
+    case DIR_MODE:
+      return info.listStatus(upperPath, info.lowerTargetPath);
+    case LEAF_FILE:
+    case SINGLEFILE_DIR:
+      return new FileStatus[] { info.getTargetFileStatus() };
+    default:
+      throw unsupported("listStatus: " + upperPath);
+    }
+  }
+
+  @Override
+  public void setWorkingDirectory(Path new_dir) {
+    workDir = new_dir;
+  }
+
+  @Override
+  public Path getWorkingDirectory() {
+    return workDir;
+  }
+
+  public FileStatus makeFileStatus(Path upperPath, Path lowerPath) throws IOException {
+    FileStatus status = lowerPath.getFileSystem(conf).getFileStatus(lowerPath);
+    status = new FileStatus(status);
+    status.setPath(upperPath);
+    return status;
+  }
+
+  private static FileStatus makeDirFileStatus(FileStatus lowerStatus) throws IOException {
+    return makeDirFileStatus(makeSfsPath(lowerStatus.getPath()), lowerStatus);
+  }
+
+  private static Path makeSfsPath(Path path) throws IOException {
+    URI oldUri = path.toUri();
+    if (oldUri.getScheme().startsWith("sfs+")) {
+      throw new IOException("unexpected path");
+    }
+    return new Path("sfs+" + oldUri.getScheme(), oldUri.getAuthority(), oldUri.getPath());
+  }
+
+  public FileStatus makeDirFileStatus(Path upperPath, Path lowerPath) throws IOException {
+    FileStatus status = lowerPath.getFileSystem(conf).getFileStatus(lowerPath);
+    return makeDirFileStatus(upperPath, status);
+  }
+
+  public static FileStatus makeDirFileStatus(Path upperPath, FileStatus status) throws IOException {
+    FileStatus newStatus = new FileStatus(status.getLen(), true, status.getReplication(), status.getBlockSize(),
+        status.getModificationTime(), status.getAccessTime(), addExecute(status.getPermission()), status.getOwner(),
+        status.getGroup(), (status.isSymlink() ? status.getSymlink() : null), status.getPath());
+    newStatus.setPath(upperPath);
+    return newStatus;
+  }
+
+  private static FsPermission addExecute(FsPermission permission) {
+    return new FsPermission(permission.toShort() | 1 | (1 << 3) | (1 << 6));
+  }
+
+  @Override
+  public FSDataOutputStream create(Path f, FsPermission permission, boolean overwrite, int bufferSize,
+      short replication, long blockSize, Progressable progress) throws IOException {
+    throw unsupported("create: " + f);
+  }
+
+  @Override
+  public FSDataOutputStream append(Path f, int bufferSize, Progressable progress) throws IOException {
+    throw unsupported("append: " + f);
+
+  }
+
+  @Override
+  public boolean rename(Path src, Path dst) throws IOException {
+    throw unsupported("rename: " + src + " to " + dst);
+  }
+
+  @Override
+  public boolean delete(Path f, boolean recursive) throws IOException {
+    throw unsupported("delete: " + f);
+
+  }
+
+  @Override
+  public boolean mkdirs(Path f, FsPermission permission) throws IOException {
+    throw unsupported("mkdirs: " + f);
+  }
+
+  private IOException unsupported(String str) {
+    return new IOException("Unsupported SFS filesystem operation! (" + str + ")");
+  }
+
+  public String getCanonicalServiceName() {

Review comment:
       override

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/io/SingleFileSystem.java
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.io;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Implements an abstraction layer to show files in a single directory.
+ *
+ * Suppose the filesystem has a directory in which there are multiple files:
+ * file://somedir/f1.txt
+ * file://somedir/f2.txt
+ *
+ * In case of Hive the content of a directory may be inside a table.
+ * To give a way to show a single file as a single file in a directory it could be specified:
+ *
+ * sfs+file://somedir/f1.txt/#SINGLEFILE#
+ *
+ * This will be a directory containing only the f1.txt and nothing else.
+ *
+ */
+public abstract class SingleFileSystem extends FileSystem {
+
+  public static class HDFS extends SingleFileSystem {
+  }
+
+  public static class S3A extends SingleFileSystem {
+  }
+
+  public static class ABFS extends SingleFileSystem {
+  }
+
+  public static class ABFSS extends SingleFileSystem {
+  }
+
+  public static class ADL extends SingleFileSystem {
+  }
+
+  public static class GS extends SingleFileSystem {
+  }
+
+  public static class O3FS extends SingleFileSystem {
+  }
+
+  public static class PFILE extends SingleFileSystem {
+  }
+
+  public static class FILE extends SingleFileSystem {
+  }
+
+  private static final String SINGLEFILE = "#SINGLEFILE#";
+
+  private URI uri;
+  private Configuration conf;
+  private Path workDir;
+
+  public String getScheme() {
+    return "sfs+" + getClass().getSimpleName().toLowerCase();
+  }
+
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    this.uri = uri;
+    this.conf = conf;
+  }
+
+  @Override
+  public URI getUri() {
+    return uri;
+  }
+
+  @Override
+  public FSDataInputStream open(Path upperPath, int bufferSize) throws IOException {
+    SfsInfo info = new SfsInfo(upperPath);
+    switch (info.type) {
+    case LEAF_FILE:
+      return info.lowerTargetPath.getFileSystem(conf).open(info.lowerTargetPath, bufferSize);
+    default:
+      throw unsupported("open:" + upperPath);
+    }
+  }
+
+  @Override
+  public FileStatus getFileStatus(Path upperPath) throws IOException {
+    SfsInfo info = new SfsInfo(upperPath);
+    switch (info.type) {
+    case LEAF_FILE:
+      return info.getTargetFileStatus();
+    case DIR_MODE:
+    case SINGLEFILE_DIR:
+      return makeDirFileStatus(upperPath, info.lowerTargetPath);
+    default:
+      throw unsupported("fileStatus:" + upperPath);
+    }
+  }
+
+  enum SfsInodeType {
+    LEAF_FILE, SINGLEFILE_DIR, DIR_MODE, NONEXISTENT,
+  }
+
+  class SfsInfo {
+
+    private String[] parts;
+    private URI uri;
+    private SfsInodeType type;
+    private Path lowerTargetPath;

Review comment:
       apidoc:
   
   `sfs+file:///dir/d1.txt/#SINGLEFILE#`
   
   lower: `file:///dir/d1.txt`
   upper: `sfs+file:///dir/d1.txt/#SINGLEFILE#/d1.txt`
   
   

##########
File path: ql/src/test/org/apache/hadoop/hive/ql/io/TestSingleFileSystem.java
##########
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.io;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Scanner;
+import java.util.ServiceLoader;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+public class TestSingleFileSystem {
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private File f1;
+  private String f1path;
+  private FileSystem fs;
+
+  @Test
+  public void testAllRegistered() {
+    Class<SingleFileSystem> c = SingleFileSystem.class;

Review comment:
       inline

##########
File path: ql/src/java/org/apache/hadoop/hive/ql/io/SingleFileSystem.java
##########
@@ -0,0 +1,293 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.io;
+
+import java.io.FileNotFoundException;
+import java.io.IOException;
+import java.net.URI;
+import java.util.ArrayList;
+import java.util.List;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FSDataOutputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.apache.hadoop.fs.permission.FsPermission;
+import org.apache.hadoop.util.Progressable;
+
+/**
+ * Implements an abstraction layer to show files in a single directory.
+ *
+ * Suppose the filesystem has a directory in which there are multiple files:
+ * file://somedir/f1.txt
+ * file://somedir/f2.txt
+ *
+ * In case of Hive the content of a directory may be inside a table.
+ * To give a way to show a single file as a single file in a directory it could be specified:
+ *
+ * sfs+file://somedir/f1.txt/#SINGLEFILE#
+ *
+ * This will be a directory containing only the f1.txt and nothing else.
+ *
+ */
+public abstract class SingleFileSystem extends FileSystem {
+
+  public static class HDFS extends SingleFileSystem {
+  }
+
+  public static class S3A extends SingleFileSystem {
+  }
+
+  public static class ABFS extends SingleFileSystem {
+  }
+
+  public static class ABFSS extends SingleFileSystem {
+  }
+
+  public static class ADL extends SingleFileSystem {
+  }
+
+  public static class GS extends SingleFileSystem {
+  }
+
+  public static class O3FS extends SingleFileSystem {
+  }
+
+  public static class PFILE extends SingleFileSystem {
+  }
+
+  public static class FILE extends SingleFileSystem {
+  }
+
+  private static final String SINGLEFILE = "#SINGLEFILE#";
+
+  private URI uri;
+  private Configuration conf;
+  private Path workDir;
+
+  public String getScheme() {
+    return "sfs+" + getClass().getSimpleName().toLowerCase();
+  }
+
+  @Override
+  public void initialize(URI uri, Configuration conf) throws IOException {
+    super.initialize(uri, conf);
+    this.uri = uri;
+    this.conf = conf;
+  }
+
+  @Override
+  public URI getUri() {
+    return uri;
+  }
+
+  @Override
+  public FSDataInputStream open(Path upperPath, int bufferSize) throws IOException {

Review comment:
       lower/upprer

##########
File path: ql/src/test/org/apache/hadoop/hive/ql/io/TestSingleFileSystem.java
##########
@@ -0,0 +1,160 @@
+/*
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *     http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing, software
+ * distributed under the License is distributed on an "AS IS" BASIS,
+ * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
+ * See the License for the specific language governing permissions and
+ * limitations under the License.
+ */
+
+package org.apache.hadoop.hive.ql.io;
+
+import static org.junit.Assert.assertEquals;
+import static org.junit.Assert.assertTrue;
+import static org.junit.Assert.fail;
+
+import java.io.File;
+import java.io.IOException;
+import java.util.HashSet;
+import java.util.Scanner;
+import java.util.ServiceLoader;
+import java.util.Set;
+
+import org.apache.hadoop.conf.Configuration;
+import org.apache.hadoop.fs.FSDataInputStream;
+import org.apache.hadoop.fs.FileStatus;
+import org.apache.hadoop.fs.FileSystem;
+import org.apache.hadoop.fs.Path;
+import org.junit.Before;
+import org.junit.Rule;
+import org.junit.Test;
+import org.junit.rules.TemporaryFolder;
+
+import com.google.common.base.Charsets;
+import com.google.common.io.Files;
+
+public class TestSingleFileSystem {
+
+  @Rule
+  public TemporaryFolder folder = new TemporaryFolder();
+  private File f1;
+  private String f1path;
+  private FileSystem fs;
+
+  @Test
+  public void testAllRegistered() {
+    Class<SingleFileSystem> c = SingleFileSystem.class;
+    ServiceLoader<FileSystem> fs = ServiceLoader.load(FileSystem.class);
+    Set<Class<?>> knownFileSystems = new HashSet<>();
+
+    for (FileSystem fileSystem : fs) {
+      knownFileSystems.add(fileSystem.getClass());
+    }
+
+    for (Class<?> sfsClass : c.getDeclaredClasses()) {
+      if (c.isAssignableFrom(sfsClass)) {
+        if (!knownFileSystems.contains(sfsClass)) {
+          fail(sfsClass + " is not registered!");
+        }
+      }
+    }
+  }
+
+  @Before
+  public void before() throws Exception {
+    f1 = folder.newFile("f1");
+    Files.write("asd", f1, Charsets.ISO_8859_1);
+    f1path = f1.toURI().toString();
+    Path p = new Path("sfs+" + f1path);
+    fs = p.getFileSystem(new Configuration());
+  }
+
+  @Test
+  public void testGetFileStatus() throws Exception {
+    assertSfsDir(fs.getFileStatus(new Path("sfs+" + folder.getRoot().toURI())));
+    assertSfsDir(fs.getFileStatus(new Path("sfs+" + f1path)));
+    assertSfsDir(fs.getFileStatus(new Path("sfs+" + f1path + "/#SINGLEFILE#")));
+    assertSfsFile(fs.getFileStatus(new Path("sfs+" + f1path + "/#SINGLEFILE#/f1")));
+  }
+
+  @Test
+  public void testListStatusSingleFileDir() throws Exception {
+    String targetSfsPath = "sfs+" + f1path + "/#SINGLEFILE#";
+    FileStatus[] list = fs.listStatus(new Path(targetSfsPath));
+    assertEquals(1, list.length);
+    assertEquals(targetSfsPath + "/f1", list[0].getPath().toString());
+  }
+
+  @Test
+  public void testListStatusSingleFileDir2() throws Exception {
+    String targetSfsPath = "sfs+" + f1path + "/#SINGLEFILE#/";

Review comment:
       add slash to testname

##########
File path: ql/src/test/queries/clientpositive/sfs.q
##########
@@ -0,0 +1,19 @@
+
+dfs ${system:test.dfs.mkdir} -p file:///${system:test.tmp.dir}/sfs;
+dfs -cp ${system:hive.root}/data/files/table1_delim.txt file:///${system:test.tmp.dir}/sfs/f1.txt;
+dfs -cp ${system:hive.root}/data/files/table1_delim.txt file:///${system:test.tmp.dir}/sfs/f2.txt;
+
+create external table t1 (a string,b string,c string) location 'file://${system:test.tmp.dir}/sfs';
+
+select * from t1;
+
+create external table t1s (a string,b string,c string) location 'sfs+file://${system:test.tmp.dir}/sfs/f1.txt/#SINGLEFILE#';
+-- create external table t1s (a string,b string,c string) location 'sfs:///pfile/${system:test.tmp.dir}/sfs/f1.txt/#SINGLEFILE#';

Review comment:
       cleanup




-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] kgyrtkirk merged pull request #2680: HIVE-25569: Enable table definition over a single file(SFS)

Posted by GitBox <gi...@apache.org>.
kgyrtkirk merged pull request #2680:
URL: https://github.com/apache/hive/pull/2680


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org


[GitHub] [hive] kgyrtkirk merged pull request #2680: HIVE-25569: Enable table definition over a single file(SFS)

Posted by GitBox <gi...@apache.org>.
kgyrtkirk merged pull request #2680:
URL: https://github.com/apache/hive/pull/2680


   


-- 
This is an automated message from the Apache Git Service.
To respond to the message, please log on to GitHub and use the
URL above to go to the specific comment.

To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org

For queries about this service, please contact Infrastructure at:
users@infra.apache.org



---------------------------------------------------------------------
To unsubscribe, e-mail: gitbox-unsubscribe@hive.apache.org
For additional commands, e-mail: gitbox-help@hive.apache.org