1 | // Licensed to the Apache Software Foundation (ASF) under one |
2 | // or more contributor license agreements. See the NOTICE file |
3 | // distributed with this work for additional information |
4 | // regarding copyright ownership. The ASF licenses this file |
5 | // to you under the Apache License, Version 2.0 (the |
6 | // "License"); you may not use this file except in compliance |
7 | // with the License. You may obtain a copy of the License at |
8 | // |
9 | // http://www.apache.org/licenses/LICENSE-2.0 |
10 | // |
11 | // Unless required by applicable law or agreed to in writing, |
12 | // software distributed under the License is distributed on an |
13 | // "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY |
14 | // KIND, either express or implied. See the License for the |
15 | // specific language governing permissions and limitations |
16 | // under the License. |
17 | |
18 | #ifndef ARROW_IO_READAHEAD_H |
19 | #define ARROW_IO_READAHEAD_H |
20 | |
21 | #include <cstdint> |
22 | #include <memory> |
23 | |
24 | #include "arrow/util/visibility.h" |
25 | |
26 | namespace arrow { |
27 | |
28 | class MemoryPool; |
29 | class ResizableBuffer; |
30 | class Status; |
31 | |
32 | namespace io { |
33 | |
34 | class InputStream; |
35 | |
36 | namespace internal { |
37 | |
38 | struct ARROW_EXPORT ReadaheadBuffer { |
39 | std::shared_ptr<ResizableBuffer> buffer; |
40 | int64_t left_padding; |
41 | int64_t right_padding; |
42 | }; |
43 | |
44 | class ARROW_EXPORT ReadaheadSpooler { |
45 | public: |
46 | /// \brief EXPERIMENTAL: Create a readahead spooler wrapping the given input stream. |
47 | /// |
48 | /// The spooler launches a background thread that reads up to a given number |
49 | /// of fixed-size blocks in advance from the underlying stream. |
50 | /// The buffers returned by Read() will be padded at the beginning and the end |
51 | /// with the configured amount of (zeroed) bytes. |
52 | ReadaheadSpooler(MemoryPool* pool, std::shared_ptr<InputStream> raw, |
53 | int64_t read_size = kDefaultReadSize, int32_t readahead_queue_size = 1, |
54 | int64_t left_padding = 0, int64_t right_padding = 0); |
55 | |
56 | explicit ReadaheadSpooler(std::shared_ptr<InputStream> raw, |
57 | int64_t read_size = kDefaultReadSize, |
58 | int32_t readahead_queue_size = 1, int64_t left_padding = 0, |
59 | int64_t right_padding = 0); |
60 | |
61 | ~ReadaheadSpooler(); |
62 | |
63 | /// Configure zero-padding at beginning and end of buffers (default 0 bytes). |
64 | /// The buffers returned by Read() will be padded at the beginning and the end |
65 | /// with the configured amount of (zeroed) bytes. |
66 | /// Note that, as reading happens in background and in advance, changing the |
67 | /// configured values might not affect Read() results immediately. |
68 | int64_t GetLeftPadding(); |
69 | void SetLeftPadding(int64_t size); |
70 | |
71 | int64_t GetRightPadding(); |
72 | void SetRightPadding(int64_t size); |
73 | |
74 | /// \brief Close the spooler. This implicitly closes the underlying input stream. |
75 | Status Close(); |
76 | |
77 | /// \brief Read a buffer from the queue. |
78 | /// |
79 | /// If the buffer pointer in the ReadaheadBuffer is null, then EOF was |
80 | /// reached and/or the spooler was explicitly closed. |
81 | /// Otherwise, the buffer will contain at most read_size bytes in addition |
82 | /// to the configured padding (short reads are possible at the end of a file). |
83 | // How do we allow reusing the buffer in ReadaheadBuffer? perhaps by using |
84 | // a caching memory pool? |
85 | Status Read(ReadaheadBuffer* out); |
86 | |
87 | private: |
88 | static constexpr int64_t kDefaultReadSize = 1 << 20; // 1 MB |
89 | |
90 | class ARROW_NO_EXPORT Impl; |
91 | std::unique_ptr<Impl> impl_; |
92 | }; |
93 | |
94 | } // namespace internal |
95 | } // namespace io |
96 | } // namespace arrow |
97 | |
98 | #endif // ARROW_IO_READAHEAD_H |
99 | |