You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@arrow.apache.org by we...@apache.org on 2019/03/13 01:26:29 UTC

[arrow] branch master updated: ARROW-4789: [C++] Deprecate and and later remove arrow::io::ReadableFileInterface

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

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


The following commit(s) were added to refs/heads/master by this push:
     new 7a539f3  ARROW-4789: [C++] Deprecate and and later remove arrow::io::ReadableFileInterface
7a539f3 is described below

commit 7a539f3383bc6229a0a5648f1730ae75600317d8
Author: Wes McKinney <we...@apache.org>
AuthorDate: Tue Mar 12 20:26:19 2019 -0500

    ARROW-4789: [C++] Deprecate and and later remove arrow::io::ReadableFileInterface
    
    Author: Wes McKinney <we...@apache.org>
    Author: Krisztián Szűcs <sz...@gmail.com>
    
    Closes #3831 from kszucs/ARROW-4789 and squashes the following commits:
    
    c48e43294 <Wes McKinney> The deprecation strategy doesn't seem to work with MSVC, so exclude (will have to be good enough)
    227428318 <Wes McKinney> Fix usages of now-deprecated ReadableFileInterface API
    dc29c29ef <Krisztián Szűcs> deprecate WriteableFile and ReadableFileInterface
---
 cpp/src/arrow/adapters/orc/adapter.cc |  8 ++++----
 cpp/src/arrow/adapters/orc/adapter.h  |  4 ++--
 cpp/src/arrow/io/interfaces.h         | 11 ++++++++---
 cpp/src/parquet/arrow/reader.cc       |  4 ++--
 cpp/src/parquet/arrow/reader.h        |  4 ++--
 cpp/src/parquet/file_reader.cc        |  6 +++---
 cpp/src/parquet/file_reader.h         |  4 ++--
 cpp/src/parquet/reader-test.cc        |  4 ++--
 cpp/src/parquet/util/memory.cc        |  3 +--
 cpp/src/parquet/util/memory.h         |  7 +++----
 10 files changed, 29 insertions(+), 26 deletions(-)

diff --git a/cpp/src/arrow/adapters/orc/adapter.cc b/cpp/src/arrow/adapters/orc/adapter.cc
index 01fc09a..78a321f 100644
--- a/cpp/src/arrow/adapters/orc/adapter.cc
+++ b/cpp/src/arrow/adapters/orc/adapter.cc
@@ -68,7 +68,7 @@ namespace orc {
 
 class ArrowInputFile : public liborc::InputStream {
  public:
-  explicit ArrowInputFile(const std::shared_ptr<io::ReadableFileInterface>& file)
+  explicit ArrowInputFile(const std::shared_ptr<io::RandomAccessFile>& file)
       : file_(file) {}
 
   uint64_t getLength() const override {
@@ -95,7 +95,7 @@ class ArrowInputFile : public liborc::InputStream {
   }
 
  private:
-  std::shared_ptr<io::ReadableFileInterface> file_;
+  std::shared_ptr<io::RandomAccessFile> file_;
 };
 
 struct StripeInformation {
@@ -222,7 +222,7 @@ class ORCFileReader::Impl {
   Impl() {}
   ~Impl() {}
 
-  Status Open(const std::shared_ptr<io::ReadableFileInterface>& file, MemoryPool* pool) {
+  Status Open(const std::shared_ptr<io::RandomAccessFile>& file, MemoryPool* pool) {
     std::unique_ptr<ArrowInputFile> io_wrapper(new ArrowInputFile(file));
     liborc::ReaderOptions options;
     std::unique_ptr<liborc::Reader> liborc_reader;
@@ -682,7 +682,7 @@ ORCFileReader::ORCFileReader() { impl_.reset(new ORCFileReader::Impl()); }
 
 ORCFileReader::~ORCFileReader() {}
 
-Status ORCFileReader::Open(const std::shared_ptr<io::ReadableFileInterface>& file,
+Status ORCFileReader::Open(const std::shared_ptr<io::RandomAccessFile>& file,
                            MemoryPool* pool, std::unique_ptr<ORCFileReader>* reader) {
   auto result = std::unique_ptr<ORCFileReader>(new ORCFileReader());
   RETURN_NOT_OK(result->impl_->Open(file, pool));
diff --git a/cpp/src/arrow/adapters/orc/adapter.h b/cpp/src/arrow/adapters/orc/adapter.h
index f482dee..5ae19b2 100644
--- a/cpp/src/arrow/adapters/orc/adapter.h
+++ b/cpp/src/arrow/adapters/orc/adapter.h
@@ -47,8 +47,8 @@ class ARROW_EXPORT ORCFileReader {
   /// \param[in] pool a MemoryPool to use for buffer allocations
   /// \param[out] reader the returned reader object
   /// \return Status
-  static Status Open(const std::shared_ptr<io::ReadableFileInterface>& file,
-                     MemoryPool* pool, std::unique_ptr<ORCFileReader>* reader);
+  static Status Open(const std::shared_ptr<io::RandomAccessFile>& file, MemoryPool* pool,
+                     std::unique_ptr<ORCFileReader>* reader);
 
   /// \brief Return the schema read from the ORC file
   ///
diff --git a/cpp/src/arrow/io/interfaces.h b/cpp/src/arrow/io/interfaces.h
index 7104aff..19cd2b5 100644
--- a/cpp/src/arrow/io/interfaces.h
+++ b/cpp/src/arrow/io/interfaces.h
@@ -182,15 +182,20 @@ class ARROW_EXPORT WritableFile : public OutputStream, public Seekable {
   WritableFile() = default;
 };
 
-// TODO(wesm): remove this after 0.11
-using WriteableFile = WritableFile;
-
 class ARROW_EXPORT ReadWriteFileInterface : public RandomAccessFile, public WritableFile {
  protected:
   ReadWriteFileInterface() { RandomAccessFile::set_mode(FileMode::READWRITE); }
 };
 
+// TODO(kszucs): remove this after 0.13
+#ifndef _MSC_VER
+using WriteableFile ARROW_DEPRECATED("Use WritableFile") = WritableFile;
+using ReadableFileInterface ARROW_DEPRECATED("Use RandomAccessFile") = RandomAccessFile;
+#else
+// MSVC does not like using ARROW_DEPRECATED with using declarations
+using WriteableFile = WritableFile;
 using ReadableFileInterface = RandomAccessFile;
+#endif
 
 }  // namespace io
 }  // namespace arrow
diff --git a/cpp/src/parquet/arrow/reader.cc b/cpp/src/parquet/arrow/reader.cc
index b36a336..61f5bb2 100644
--- a/cpp/src/parquet/arrow/reader.cc
+++ b/cpp/src/parquet/arrow/reader.cc
@@ -665,7 +665,7 @@ Status FileReader::Impl::ReadRowGroup(int i, std::shared_ptr<Table>* table) {
 }
 
 // Static ctor
-Status OpenFile(const std::shared_ptr<::arrow::io::ReadableFileInterface>& file,
+Status OpenFile(const std::shared_ptr<::arrow::io::RandomAccessFile>& file,
                 MemoryPool* allocator, const ReaderProperties& props,
                 const std::shared_ptr<FileMetaData>& metadata,
                 std::unique_ptr<FileReader>* reader) {
@@ -677,7 +677,7 @@ Status OpenFile(const std::shared_ptr<::arrow::io::ReadableFileInterface>& file,
   return Status::OK();
 }
 
-Status OpenFile(const std::shared_ptr<::arrow::io::ReadableFileInterface>& file,
+Status OpenFile(const std::shared_ptr<::arrow::io::RandomAccessFile>& file,
                 MemoryPool* allocator, std::unique_ptr<FileReader>* reader) {
   return OpenFile(file, allocator, ::parquet::default_reader_properties(), nullptr,
                   reader);
diff --git a/cpp/src/parquet/arrow/reader.h b/cpp/src/parquet/arrow/reader.h
index 5286e74..7ef21fd 100644
--- a/cpp/src/parquet/arrow/reader.h
+++ b/cpp/src/parquet/arrow/reader.h
@@ -295,14 +295,14 @@ class PARQUET_EXPORT ColumnReader {
 //
 // metadata : separately-computed file metadata, can be nullptr
 PARQUET_EXPORT
-::arrow::Status OpenFile(const std::shared_ptr<::arrow::io::ReadableFileInterface>& file,
+::arrow::Status OpenFile(const std::shared_ptr<::arrow::io::RandomAccessFile>& file,
                          ::arrow::MemoryPool* allocator,
                          const ReaderProperties& properties,
                          const std::shared_ptr<FileMetaData>& metadata,
                          std::unique_ptr<FileReader>* reader);
 
 PARQUET_EXPORT
-::arrow::Status OpenFile(const std::shared_ptr<::arrow::io::ReadableFileInterface>& file,
+::arrow::Status OpenFile(const std::shared_ptr<::arrow::io::RandomAccessFile>& file,
                          ::arrow::MemoryPool* allocator,
                          std::unique_ptr<FileReader>* reader);
 
diff --git a/cpp/src/parquet/file_reader.cc b/cpp/src/parquet/file_reader.cc
index 0f8e359..915fd54 100644
--- a/cpp/src/parquet/file_reader.cc
+++ b/cpp/src/parquet/file_reader.cc
@@ -246,7 +246,7 @@ std::unique_ptr<ParquetFileReader::Contents> ParquetFileReader::Contents::Open(
 }
 
 std::unique_ptr<ParquetFileReader> ParquetFileReader::Open(
-    const std::shared_ptr<::arrow::io::ReadableFileInterface>& source,
+    const std::shared_ptr<::arrow::io::RandomAccessFile>& source,
     const ReaderProperties& props, const std::shared_ptr<FileMetaData>& metadata) {
   std::unique_ptr<RandomAccessSource> io_wrapper(new ArrowInputFile(source));
   return Open(std::move(io_wrapper), props, metadata);
@@ -264,7 +264,7 @@ std::unique_ptr<ParquetFileReader> ParquetFileReader::Open(
 std::unique_ptr<ParquetFileReader> ParquetFileReader::OpenFile(
     const std::string& path, bool memory_map, const ReaderProperties& props,
     const std::shared_ptr<FileMetaData>& metadata) {
-  std::shared_ptr<::arrow::io::ReadableFileInterface> source;
+  std::shared_ptr<::arrow::io::RandomAccessFile> source;
   if (memory_map) {
     std::shared_ptr<::arrow::io::MemoryMappedFile> handle;
     PARQUET_THROW_NOT_OK(
@@ -305,7 +305,7 @@ std::shared_ptr<RowGroupReader> ParquetFileReader::RowGroup(int i) {
 // File metadata helpers
 
 std::shared_ptr<FileMetaData> ReadMetaData(
-    const std::shared_ptr<::arrow::io::ReadableFileInterface>& source) {
+    const std::shared_ptr<::arrow::io::RandomAccessFile>& source) {
   return ParquetFileReader::Open(source)->metadata();
 }
 
diff --git a/cpp/src/parquet/file_reader.h b/cpp/src/parquet/file_reader.h
index 2d1cc92..faaf44e 100644
--- a/cpp/src/parquet/file_reader.h
+++ b/cpp/src/parquet/file_reader.h
@@ -100,7 +100,7 @@ class PARQUET_EXPORT ParquetFileReader {
   // Create a file reader instance from an Arrow file object. Thread-safety is
   // the responsibility of the file implementation
   static std::unique_ptr<ParquetFileReader> Open(
-      const std::shared_ptr<::arrow::io::ReadableFileInterface>& source,
+      const std::shared_ptr<::arrow::io::RandomAccessFile>& source,
       const ReaderProperties& props = default_reader_properties(),
       const std::shared_ptr<FileMetaData>& metadata = NULLPTR);
 
@@ -127,7 +127,7 @@ class PARQUET_EXPORT ParquetFileReader {
 
 // Read only Parquet file metadata
 std::shared_ptr<FileMetaData> PARQUET_EXPORT
-ReadMetaData(const std::shared_ptr<::arrow::io::ReadableFileInterface>& source);
+ReadMetaData(const std::shared_ptr<::arrow::io::RandomAccessFile>& source);
 
 /// \brief Scan all values in file. Useful for performance testing
 /// \param[in] columns the column numbers to scan. If empty scans all
diff --git a/cpp/src/parquet/reader-test.cc b/cpp/src/parquet/reader-test.cc
index a0536b5..63d394e 100644
--- a/cpp/src/parquet/reader-test.cc
+++ b/cpp/src/parquet/reader-test.cc
@@ -190,8 +190,8 @@ class TestLocalFile : public ::testing::Test {
 
 class HelperFileClosed : public ArrowInputFile {
  public:
-  explicit HelperFileClosed(
-      const std::shared_ptr<::arrow::io::ReadableFileInterface>& file, bool* close_called)
+  explicit HelperFileClosed(const std::shared_ptr<::arrow::io::RandomAccessFile>& file,
+                            bool* close_called)
       : ArrowInputFile(file), close_called_(close_called) {}
 
   void Close() override { *close_called_ = true; }
diff --git a/cpp/src/parquet/util/memory.cc b/cpp/src/parquet/util/memory.cc
index b3f83bd..e37fd88 100644
--- a/cpp/src/parquet/util/memory.cc
+++ b/cpp/src/parquet/util/memory.cc
@@ -130,8 +130,7 @@ int64_t ArrowFileMethods::Tell() {
   return position;
 }
 
-ArrowInputFile::ArrowInputFile(
-    const std::shared_ptr<::arrow::io::ReadableFileInterface>& file)
+ArrowInputFile::ArrowInputFile(const std::shared_ptr<::arrow::io::RandomAccessFile>& file)
     : file_(file) {}
 
 ::arrow::io::FileInterface* ArrowInputFile::file_interface() { return file_.get(); }
diff --git a/cpp/src/parquet/util/memory.h b/cpp/src/parquet/util/memory.h
index d63ed84..02cf082 100644
--- a/cpp/src/parquet/util/memory.h
+++ b/cpp/src/parquet/util/memory.h
@@ -138,8 +138,7 @@ class PARQUET_EXPORT ArrowFileMethods : virtual public FileInterface {
 /// This interface depends on the threadsafety of the underlying Arrow file interface
 class PARQUET_EXPORT ArrowInputFile : public ArrowFileMethods, public RandomAccessSource {
  public:
-  explicit ArrowInputFile(
-      const std::shared_ptr<::arrow::io::ReadableFileInterface>& file);
+  explicit ArrowInputFile(const std::shared_ptr<::arrow::io::RandomAccessFile>& file);
 
   int64_t Size() const override;
 
@@ -153,7 +152,7 @@ class PARQUET_EXPORT ArrowInputFile : public ArrowFileMethods, public RandomAcce
   /// Returns bytes read
   int64_t ReadAt(int64_t position, int64_t nbytes, uint8_t* out) override;
 
-  std::shared_ptr<::arrow::io::ReadableFileInterface> file() const { return file_; }
+  std::shared_ptr<::arrow::io::RandomAccessFile> file() const { return file_; }
 
   // Diamond inheritance
   using ArrowFileMethods::Close;
@@ -161,7 +160,7 @@ class PARQUET_EXPORT ArrowInputFile : public ArrowFileMethods, public RandomAcce
 
  private:
   ::arrow::io::FileInterface* file_interface() override;
-  std::shared_ptr<::arrow::io::ReadableFileInterface> file_;
+  std::shared_ptr<::arrow::io::RandomAccessFile> file_;
 };
 
 class PARQUET_EXPORT ArrowOutputStream : public ArrowFileMethods, public OutputStream {