You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@iceberg.apache.org by bl...@apache.org on 2019/09/06 20:26:23 UTC

[incubator-iceberg] branch master updated: Throw NotFoundException from IO methods when file does not exist (#454)

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

blue pushed a commit to branch master
in repository https://gitbox.apache.org/repos/asf/incubator-iceberg.git


The following commit(s) were added to refs/heads/master by this push:
     new 3e8a092  Throw NotFoundException from IO methods when file does not exist (#454)
3e8a092 is described below

commit 3e8a0929e0fdd1b7e801473d43aea0312f3fa6c3
Author: John Zhuge <jz...@apache.org>
AuthorDate: Fri Sep 6 13:26:19 2019 -0700

    Throw NotFoundException from IO methods when file does not exist (#454)
---
 api/src/main/java/org/apache/iceberg/Files.java    |  3 +-
 .../iceberg/exceptions/NotFoundException.java      | 33 ++++++++++++++++++++++
 .../main/java/org/apache/iceberg/io/InputFile.java |  2 ++
 .../org/apache/iceberg/hadoop/HadoopInputFile.java |  4 +++
 4 files changed, 41 insertions(+), 1 deletion(-)

diff --git a/api/src/main/java/org/apache/iceberg/Files.java b/api/src/main/java/org/apache/iceberg/Files.java
index 4e570ad..6489e9d 100644
--- a/api/src/main/java/org/apache/iceberg/Files.java
+++ b/api/src/main/java/org/apache/iceberg/Files.java
@@ -25,6 +25,7 @@ import java.io.IOException;
 import java.io.RandomAccessFile;
 import java.nio.file.Paths;
 import org.apache.iceberg.exceptions.AlreadyExistsException;
+import org.apache.iceberg.exceptions.NotFoundException;
 import org.apache.iceberg.exceptions.RuntimeIOException;
 import org.apache.iceberg.io.InputFile;
 import org.apache.iceberg.io.OutputFile;
@@ -124,7 +125,7 @@ public class Files {
       try {
         return new SeekableFileInputStream(new RandomAccessFile(file, "r"));
       } catch (FileNotFoundException e) {
-        throw new RuntimeIOException(e, "Failed to read file: %s", file);
+        throw new NotFoundException(e, "Failed to read file: %s", file);
       }
     }
 
diff --git a/api/src/main/java/org/apache/iceberg/exceptions/NotFoundException.java b/api/src/main/java/org/apache/iceberg/exceptions/NotFoundException.java
new file mode 100644
index 0000000..a2d05dd
--- /dev/null
+++ b/api/src/main/java/org/apache/iceberg/exceptions/NotFoundException.java
@@ -0,0 +1,33 @@
+/*
+ * 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.iceberg.exceptions;
+
+/**
+ * Exception raised when attempting to read a file that does not exist.
+ */
+public class NotFoundException extends RuntimeException {
+  public NotFoundException(String message, Object... args) {
+    super(String.format(message, args));
+  }
+
+  public NotFoundException(Throwable cause, String message, Object... args) {
+    super(String.format(message, args), cause);
+  }
+}
diff --git a/api/src/main/java/org/apache/iceberg/io/InputFile.java b/api/src/main/java/org/apache/iceberg/io/InputFile.java
index 7d21f73..70b1e9f 100644
--- a/api/src/main/java/org/apache/iceberg/io/InputFile.java
+++ b/api/src/main/java/org/apache/iceberg/io/InputFile.java
@@ -20,6 +20,7 @@
 package org.apache.iceberg.io;
 
 import java.io.IOException;
+import org.apache.iceberg.exceptions.NotFoundException;
 import org.apache.iceberg.exceptions.RuntimeIOException;
 
 /**
@@ -38,6 +39,7 @@ public interface InputFile {
    * Opens a new {@link SeekableInputStream} for the underlying data file
    *
    * @return a seekable stream for reading the file
+   * @throws NotFoundException If the file does not exist
    * @throws RuntimeIOException If the implementation throws an {@link IOException}
    */
   SeekableInputStream newStream();
diff --git a/core/src/main/java/org/apache/iceberg/hadoop/HadoopInputFile.java b/core/src/main/java/org/apache/iceberg/hadoop/HadoopInputFile.java
index df745ca..00b43ad 100644
--- a/core/src/main/java/org/apache/iceberg/hadoop/HadoopInputFile.java
+++ b/core/src/main/java/org/apache/iceberg/hadoop/HadoopInputFile.java
@@ -19,11 +19,13 @@
 
 package org.apache.iceberg.hadoop;
 
+import java.io.FileNotFoundException;
 import java.io.IOException;
 import org.apache.hadoop.conf.Configuration;
 import org.apache.hadoop.fs.FileStatus;
 import org.apache.hadoop.fs.FileSystem;
 import org.apache.hadoop.fs.Path;
+import org.apache.iceberg.exceptions.NotFoundException;
 import org.apache.iceberg.exceptions.RuntimeIOException;
 import org.apache.iceberg.io.InputFile;
 import org.apache.iceberg.io.SeekableInputStream;
@@ -123,6 +125,8 @@ public class HadoopInputFile implements InputFile {
   public SeekableInputStream newStream() {
     try {
       return HadoopStreams.wrap(fs.open(path));
+    } catch (FileNotFoundException e) {
+      throw new NotFoundException(e, "Failed to open input stream for file: %s", path);
     } catch (IOException e) {
       throw new RuntimeIOException(e, "Failed to open input stream for file: %s", path);
     }