You are viewing a plain text version of this content. The canonical link for it is here.
Posted to issues@orc.apache.org by GitBox <gi...@apache.org> on 2022/10/06 13:57:27 UTC

[GitHub] [orc] coderex2522 opened a new pull request, #1271: ORC-1280:[C++][Part I] Implement block-based buffer

coderex2522 opened a new pull request, #1271:
URL: https://github.com/apache/orc/pull/1271

   ### What changes were proposed in this pull request?
   This patch implements a block-based buffer allocator. Provides smarter memory management for  the buffer of BufferedOutputStream class.
   
   ### Why are the changes needed?
   This patch implements smarter memory management, which can effectively solve the [issue-1240](https://github.com/apache/orc/issues/1240). And replacing DataBuffer with BlockBuffer in BufferedOutputStream class will be implemented in another patch.
   
   ### How was this patch tested?
   The UTs in TestBlockBuffer.cc can test this patch.
   


-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] wgtmac commented on pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
wgtmac commented on PR #1271:
URL: https://github.com/apache/orc/pull/1271#issuecomment-1275475996

   I've submitted and closed it. Thanks @coderex2522 and @dongjoon-hyun 


-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] wgtmac merged pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
wgtmac merged PR #1271:
URL: https://github.com/apache/orc/pull/1271


-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r989818621


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,90 @@
+

Review Comment:
   Done.



##########
c++/test/TestBlockBuffer.cc:
##########
@@ -0,0 +1,82 @@
+

Review Comment:
   Done



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990601575


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() = default;
+  };
+
+  /**
+   * BlockBuffer implements a memory allocation policy based on
+   * equal-length blocks. BlockBuffer will reserve multiple blocks
+   * for allocation.
+   */
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the allocated block object.
+     * The last allocated block size may be less than blockSize,
+     * and the rest of the blocks are all of size blockSize.
+     * @param blockIndex the index of blocks
+     * @return the allocated block object
+     */
+    Block getBlock(uint64_t blockIndex) const;

Review Comment:
   Since the struct Block contains bare data pointer, user can always write the memory even if it is const Block.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990740765


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() = default;
+  };
+
+  /**
+   * BlockBuffer implements a memory allocation policy based on
+   * equal-length blocks. BlockBuffer will reserve multiple blocks
+   * for allocation.
+   */
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the allocated block object.
+     * The last allocated block size may be less than blockSize,
+     * and the rest of the blocks are all of size blockSize.
+     * @param blockIndex the index of blocks
+     * @return the allocated block object
+     */
+    Block getBlock(uint64_t blockIndex) const;
+
+    /**
+     * Get a empty block or a new block if the buffer is exhausted.
+     * If the last allocated block size is less than blockSize,
+     * the empty block size is blockSize - lastBlockSize.
+     * Otherwise, the empty block size is blockSize.
+     * @return a empty block object
+     */
+    Block getEmptyBlock();
+
+    /**
+     * Get the number of allocated blocks
+     */
+    uint64_t getBlockNumber() const {
+      return (currentSize + blockSize - 1) / blockSize;
+    }
+
+    uint64_t size() const {
+      return currentSize;
+    }
+
+    uint64_t capacity() const {
+      return currentCapacity;
+    }
+
+    void resize(uint64_t size);

Review Comment:
   Class DataBuffer only recycle memory during destructor, so class BlockBuffer does not support shrink function, which can be further optimized later.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r989757028


##########
c++/src/MemoryPool.hh:
##########
@@ -0,0 +1,92 @@
+
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"

Review Comment:
   Sure, I rename src/MemoryPool.hh into src/BlockBuffer.hh.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] dongjoon-hyun commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r989734961


##########
c++/test/TestBlockBuffer.cc:
##########
@@ -0,0 +1,82 @@
+

Review Comment:
   Redundant space.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] wgtmac commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990591121


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() = default;
+  };
+
+  /**
+   * BlockBuffer implements a memory allocation policy based on
+   * equal-length blocks. BlockBuffer will reserve multiple blocks
+   * for allocation.
+   */
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the allocated block object.
+     * The last allocated block size may be less than blockSize,
+     * and the rest of the blocks are all of size blockSize.
+     * @param blockIndex the index of blocks
+     * @return the allocated block object
+     */
+    Block getBlock(uint64_t blockIndex) const;
+
+    /**
+     * Get a empty block or a new block if the buffer is exhausted.
+     * If the last allocated block size is less than blockSize,
+     * the empty block size is blockSize - lastBlockSize.

Review Comment:
   Where is the lastBlockSize?
   It would be more user-friendly to get the available size of the returned block via output parameter.



##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {

Review Comment:
   Is it better to make it nested class of BlockBuffer?



##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() = default;
+  };
+
+  /**
+   * BlockBuffer implements a memory allocation policy based on
+   * equal-length blocks. BlockBuffer will reserve multiple blocks
+   * for allocation.
+   */
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the allocated block object.
+     * The last allocated block size may be less than blockSize,
+     * and the rest of the blocks are all of size blockSize.
+     * @param blockIndex the index of blocks
+     * @return the allocated block object
+     */
+    Block getBlock(uint64_t blockIndex) const;
+
+    /**
+     * Get a empty block or a new block if the buffer is exhausted.
+     * If the last allocated block size is less than blockSize,
+     * the empty block size is blockSize - lastBlockSize.
+     * Otherwise, the empty block size is blockSize.
+     * @return a empty block object
+     */
+    Block getEmptyBlock();
+
+    /**
+     * Get the number of allocated blocks

Review Comment:
   Get the number of blocks that are fully or partially occupied



##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() = default;
+  };
+
+  /**
+   * BlockBuffer implements a memory allocation policy based on
+   * equal-length blocks. BlockBuffer will reserve multiple blocks
+   * for allocation.
+   */
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the allocated block object.
+     * The last allocated block size may be less than blockSize,
+     * and the rest of the blocks are all of size blockSize.
+     * @param blockIndex the index of blocks
+     * @return the allocated block object
+     */
+    Block getBlock(uint64_t blockIndex) const;
+
+    /**
+     * Get a empty block or a new block if the buffer is exhausted.
+     * If the last allocated block size is less than blockSize,
+     * the empty block size is blockSize - lastBlockSize.
+     * Otherwise, the empty block size is blockSize.
+     * @return a empty block object
+     */
+    Block getEmptyBlock();
+
+    /**
+     * Get the number of allocated blocks
+     */
+    uint64_t getBlockNumber() const {
+      return (currentSize + blockSize - 1) / blockSize;
+    }
+
+    uint64_t size() const {
+      return currentSize;
+    }
+
+    uint64_t capacity() const {
+      return currentCapacity;
+    }
+
+    void resize(uint64_t size);
+    void reserve(uint64_t capacity);

Review Comment:
   Insert a new line before reserve function. Better to explain what will happen in the reserve function



##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() = default;
+  };
+
+  /**
+   * BlockBuffer implements a memory allocation policy based on
+   * equal-length blocks. BlockBuffer will reserve multiple blocks
+   * for allocation.
+   */
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the allocated block object.
+     * The last allocated block size may be less than blockSize,
+     * and the rest of the blocks are all of size blockSize.
+     * @param blockIndex the index of blocks
+     * @return the allocated block object
+     */
+    Block getBlock(uint64_t blockIndex) const;
+
+    /**
+     * Get a empty block or a new block if the buffer is exhausted.
+     * If the last allocated block size is less than blockSize,
+     * the empty block size is blockSize - lastBlockSize.
+     * Otherwise, the empty block size is blockSize.
+     * @return a empty block object
+     */
+    Block getEmptyBlock();
+
+    /**
+     * Get the number of allocated blocks
+     */
+    uint64_t getBlockNumber() const {
+      return (currentSize + blockSize - 1) / blockSize;
+    }
+
+    uint64_t size() const {
+      return currentSize;
+    }
+
+    uint64_t capacity() const {
+      return currentCapacity;
+    }
+
+    void resize(uint64_t size);

Review Comment:
   Should we support shrinkToFit parameter?



##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() = default;
+  };
+
+  /**
+   * BlockBuffer implements a memory allocation policy based on
+   * equal-length blocks. BlockBuffer will reserve multiple blocks
+   * for allocation.
+   */
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the allocated block object.
+     * The last allocated block size may be less than blockSize,
+     * and the rest of the blocks are all of size blockSize.
+     * @param blockIndex the index of blocks
+     * @return the allocated block object
+     */
+    Block getBlock(uint64_t blockIndex) const;

Review Comment:
   User can still ruin the memory of the returned Block. Make the return type const Block?



##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() = default;
+  };
+
+  /**
+   * BlockBuffer implements a memory allocation policy based on
+   * equal-length blocks. BlockBuffer will reserve multiple blocks
+   * for allocation.
+   */
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the allocated block object.
+     * The last allocated block size may be less than blockSize,
+     * and the rest of the blocks are all of size blockSize.
+     * @param blockIndex the index of blocks
+     * @return the allocated block object
+     */
+    Block getBlock(uint64_t blockIndex) const;
+
+    /**
+     * Get a empty block or a new block if the buffer is exhausted.
+     * If the last allocated block size is less than blockSize,
+     * the empty block size is blockSize - lastBlockSize.
+     * Otherwise, the empty block size is blockSize.
+     * @return a empty block object
+     */
+    Block getEmptyBlock();

Review Comment:
   rename to getNextBlock or getBlockToWrite?



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990583906


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,89 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() {}
+  };
+
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the Block object
+     */
+    Block getBlock(uint64_t blockIndex);

Review Comment:
   Because the getBlock function does not modify class member variables, the Constant attribute is added to the function.
   



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990582975


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,89 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() {}

Review Comment:
   done.



##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,89 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() {}
+  };
+
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the Block object
+     */
+    Block getBlock(uint64_t blockIndex);
+
+    /**
+     * Get a empty block or allocate a new block if the buffer is exhausted
+     */
+    Block getEmptyBlock();
+
+    /**
+     * Get the Block number
+     */
+    uint64_t getBlockNumber() const;
+
+    uint64_t size() const {
+      return currentSize;
+    }
+
+    uint64_t capacity() const {
+      return currentCapacity;
+    }
+
+    void resize(uint64_t size);
+    void reserve(uint64_t capacity);
+  };
+} // namespace

Review Comment:
   done.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990599653


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() = default;
+  };
+
+  /**
+   * BlockBuffer implements a memory allocation policy based on
+   * equal-length blocks. BlockBuffer will reserve multiple blocks
+   * for allocation.
+   */
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the allocated block object.
+     * The last allocated block size may be less than blockSize,
+     * and the rest of the blocks are all of size blockSize.
+     * @param blockIndex the index of blocks
+     * @return the allocated block object
+     */
+    Block getBlock(uint64_t blockIndex) const;
+
+    /**
+     * Get a empty block or a new block if the buffer is exhausted.
+     * If the last allocated block size is less than blockSize,
+     * the empty block size is blockSize - lastBlockSize.
+     * Otherwise, the empty block size is blockSize.
+     * @return a empty block object
+     */
+    Block getEmptyBlock();
+
+    /**
+     * Get the number of allocated blocks
+     */
+    uint64_t getBlockNumber() const {
+      return (currentSize + blockSize - 1) / blockSize;
+    }
+
+    uint64_t size() const {
+      return currentSize;
+    }
+
+    uint64_t capacity() const {
+      return currentCapacity;
+    }
+
+    void resize(uint64_t size);
+    void reserve(uint64_t capacity);

Review Comment:
   Added some comments to explain the reserve function.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990600574


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() = default;
+  };
+
+  /**
+   * BlockBuffer implements a memory allocation policy based on
+   * equal-length blocks. BlockBuffer will reserve multiple blocks
+   * for allocation.
+   */
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the allocated block object.
+     * The last allocated block size may be less than blockSize,
+     * and the rest of the blocks are all of size blockSize.
+     * @param blockIndex the index of blocks
+     * @return the allocated block object
+     */
+    Block getBlock(uint64_t blockIndex) const;
+
+    /**
+     * Get a empty block or a new block if the buffer is exhausted.
+     * If the last allocated block size is less than blockSize,
+     * the empty block size is blockSize - lastBlockSize.

Review Comment:
   lastBlockSize means the size of the last allocated block. The comment has been modified here.
   The returned block is used to describe the section of memory that can be read or written. And I add comment to the size in struct Block. So I don't recommend adding extra output parameter.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990600082


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {

Review Comment:
   As it stands struct Block only serves class BlockBuffer, so it makes reasonable to treat it as a nested class of  class BlockBuffer.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990744453


##########
c++/src/BlockBuffer.cc:
##########
@@ -0,0 +1,85 @@
+/**
+ * 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.
+ */
+
+#include "BlockBuffer.hh"
+
+#include <algorithm>
+
+namespace orc {
+
+  BlockBuffer::BlockBuffer(MemoryPool& pool, uint64_t _blockSize)
+      : memoryPool(pool),
+        currentSize(0),
+        currentCapacity(0),
+        blockSize(_blockSize) {
+    if (blockSize == 0) {
+      throw std::logic_error("Block size cannot be zero");
+    }
+    reserve(blockSize);
+  }
+
+  BlockBuffer::~BlockBuffer() {
+    for (size_t i = 0; i < blocks.size(); ++i) {
+      memoryPool.free(blocks[i]);
+    }
+    blocks.clear();
+    currentSize = currentCapacity = 0;
+  }
+
+  BlockBuffer::Block BlockBuffer::getBlock(uint64_t blockIndex) const {
+    if (blockIndex >= getBlockNumber()) {
+      throw std::out_of_range("Block index out of range");
+    }
+    return Block(blocks[blockIndex],
+                 std::min(currentSize - blockIndex * blockSize, blockSize));
+  }
+
+  BlockBuffer::Block BlockBuffer::getNextBlock() {
+    if (currentSize < currentCapacity) {
+      Block emptyBlock(
+          blocks[currentSize / blockSize] + currentSize % blockSize,
+          blockSize - currentSize % blockSize);
+      currentSize = (currentSize / blockSize + 1) * blockSize;

Review Comment:
   Since the flush() function of class BufferedOutputStream needs to access all allocated memory blocks, it needs class BlockBuffer to provide an interface in order for BufferedOutputStream to access all allocated memory blocks.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990583906


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,89 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() {}
+  };
+
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the Block object
+     */
+    Block getBlock(uint64_t blockIndex);

Review Comment:
   Because the getBlock function does not modify class member variables, the const attribute is added to the function.
   



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] wgtmac commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990180316


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,89 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() {}
+  };
+
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the Block object
+     */
+    Block getBlock(uint64_t blockIndex);

Review Comment:
   Should we provide mutable and immutable overloads?



##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,89 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() {}
+  };
+
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the Block object
+     */
+    Block getBlock(uint64_t blockIndex);
+
+    /**
+     * Get a empty block or allocate a new block if the buffer is exhausted
+     */
+    Block getEmptyBlock();
+
+    /**
+     * Get the Block number
+     */
+    uint64_t getBlockNumber() const;
+
+    uint64_t size() const {
+      return currentSize;
+    }
+
+    uint64_t capacity() const {
+      return currentCapacity;
+    }
+
+    void resize(uint64_t size);
+    void reserve(uint64_t capacity);
+  };
+} // namespace

Review Comment:
   // namespace orc 



##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,89 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() {}
+  };
+
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the Block object
+     */
+    Block getBlock(uint64_t blockIndex);

Review Comment:
   We need some detail explanation with regard to the mechanism and behavior of the BlockBuffer. Especially concepts like block, block index and block number.



##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,89 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() {}

Review Comment:
   ~Block() = default;



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] dongjoon-hyun commented on pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on PR #1271:
URL: https://github.com/apache/orc/pull/1271#issuecomment-1270402685

   Thank you for making a PR. I had one comment about file structure, https://github.com/apache/orc/pull/1271#discussion_r989278403.
   
   cc @wgtmac , @stiga-huang 


-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990583071


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,89 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() {}
+  };
+
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the Block object
+     */
+    Block getBlock(uint64_t blockIndex);

Review Comment:
   Added some description for block, blockbuffer, block index and block number.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] wgtmac commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
wgtmac commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990733574


##########
c++/src/BlockBuffer.cc:
##########
@@ -0,0 +1,85 @@
+/**
+ * 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.
+ */
+
+#include "BlockBuffer.hh"
+
+#include <algorithm>
+
+namespace orc {
+
+  BlockBuffer::BlockBuffer(MemoryPool& pool, uint64_t _blockSize)
+      : memoryPool(pool),
+        currentSize(0),
+        currentCapacity(0),
+        blockSize(_blockSize) {
+    if (blockSize == 0) {
+      throw std::logic_error("Block size cannot be zero");
+    }
+    reserve(blockSize);
+  }
+
+  BlockBuffer::~BlockBuffer() {
+    for (size_t i = 0; i < blocks.size(); ++i) {
+      memoryPool.free(blocks[i]);
+    }
+    blocks.clear();
+    currentSize = currentCapacity = 0;
+  }
+
+  BlockBuffer::Block BlockBuffer::getBlock(uint64_t blockIndex) const {
+    if (blockIndex >= getBlockNumber()) {
+      throw std::out_of_range("Block index out of range");
+    }
+    return Block(blocks[blockIndex],
+                 std::min(currentSize - blockIndex * blockSize, blockSize));
+  }
+
+  BlockBuffer::Block BlockBuffer::getNextBlock() {
+    if (currentSize < currentCapacity) {
+      Block emptyBlock(
+          blocks[currentSize / blockSize] + currentSize % blockSize,
+          blockSize - currentSize % blockSize);
+      currentSize = (currentSize / blockSize + 1) * blockSize;

Review Comment:
   Should we provide a function to update currentSize to reflect the actual size written? Maybe something setSize() or backup(). This looks weird when the returned block is written partially meaning that currentSize is larger than used
   
   To provide better usability, we may provide a function like **void append(const char* data, size_t size)** to append data to the buffer and manage the blocks internally. This can be a separate patch.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990740765


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() = default;
+  };
+
+  /**
+   * BlockBuffer implements a memory allocation policy based on
+   * equal-length blocks. BlockBuffer will reserve multiple blocks
+   * for allocation.
+   */
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the allocated block object.
+     * The last allocated block size may be less than blockSize,
+     * and the rest of the blocks are all of size blockSize.
+     * @param blockIndex the index of blocks
+     * @return the allocated block object
+     */
+    Block getBlock(uint64_t blockIndex) const;
+
+    /**
+     * Get a empty block or a new block if the buffer is exhausted.
+     * If the last allocated block size is less than blockSize,
+     * the empty block size is blockSize - lastBlockSize.
+     * Otherwise, the empty block size is blockSize.
+     * @return a empty block object
+     */
+    Block getEmptyBlock();
+
+    /**
+     * Get the number of allocated blocks
+     */
+    uint64_t getBlockNumber() const {
+      return (currentSize + blockSize - 1) / blockSize;
+    }
+
+    uint64_t size() const {
+      return currentSize;
+    }
+
+    uint64_t capacity() const {
+      return currentCapacity;
+    }
+
+    void resize(uint64_t size);

Review Comment:
   Class DataBuffer only recycles memory during destructor, so class BlockBuffer does not support shrink function, which can be further optimized later.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] dongjoon-hyun commented on pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on PR #1271:
URL: https://github.com/apache/orc/pull/1271#issuecomment-1271182545

   It seems that the labeler is broken. Let me fix it.


-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] dongjoon-hyun commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r989278403


##########
c++/src/MemoryPool.hh:
##########
@@ -0,0 +1,92 @@
+
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"

Review Comment:
   It's a little confusing making another `MemoryPool.hh` like this. Could you create a new file instead of `MemoryPool.hh` file?



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] dongjoon-hyun commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
dongjoon-hyun commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r989734687


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,90 @@
+

Review Comment:
   Redundant space.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990600659


##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() = default;
+  };
+
+  /**
+   * BlockBuffer implements a memory allocation policy based on
+   * equal-length blocks. BlockBuffer will reserve multiple blocks
+   * for allocation.
+   */
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the allocated block object.
+     * The last allocated block size may be less than blockSize,
+     * and the rest of the blocks are all of size blockSize.
+     * @param blockIndex the index of blocks
+     * @return the allocated block object
+     */
+    Block getBlock(uint64_t blockIndex) const;
+
+    /**
+     * Get a empty block or a new block if the buffer is exhausted.
+     * If the last allocated block size is less than blockSize,
+     * the empty block size is blockSize - lastBlockSize.
+     * Otherwise, the empty block size is blockSize.
+     * @return a empty block object
+     */
+    Block getEmptyBlock();

Review Comment:
   Done. Rename to getNextBlock.



##########
c++/src/BlockBuffer.hh:
##########
@@ -0,0 +1,108 @@
+/**
+ * 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.
+ */
+
+#ifndef ORC_MEMORYPOOL_IMPL_HH
+#define ORC_MEMORYPOOL_IMPL_HH
+
+#include "orc/MemoryPool.hh"
+
+#include <vector>
+
+namespace orc {
+
+  /**
+   * Block points to a section of memory allocated by BlockBuffer,
+   * containing the corresponding physical memory address and size.
+   */
+  struct Block {
+    char* data;
+    uint64_t size;
+
+    Block() : data(nullptr), size(0) {}
+    Block(char* _data, uint64_t _size) : data(_data), size(_size) {}
+    Block(const Block& block) = default;
+    ~Block() = default;
+  };
+
+  /**
+   * BlockBuffer implements a memory allocation policy based on
+   * equal-length blocks. BlockBuffer will reserve multiple blocks
+   * for allocation.
+   */
+  class BlockBuffer {
+  private:
+    MemoryPool& memoryPool;
+    // current buffer size
+    uint64_t currentSize;
+    // maximal capacity (actual allocated memory)
+    uint64_t currentCapacity;
+    // unit for buffer expansion
+    const uint64_t blockSize;
+    // pointers to the start of each block
+    std::vector<char*> blocks;
+
+    // non-copy-constructible
+    BlockBuffer(BlockBuffer& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer& buffer) = delete;
+    BlockBuffer(BlockBuffer&& buffer) = delete;
+    BlockBuffer& operator=(BlockBuffer&& buffer) = delete;
+
+  public:
+    BlockBuffer(MemoryPool& pool, uint64_t blockSize);
+
+    ~BlockBuffer();
+
+    /**
+     * Get the allocated block object.
+     * The last allocated block size may be less than blockSize,
+     * and the rest of the blocks are all of size blockSize.
+     * @param blockIndex the index of blocks
+     * @return the allocated block object
+     */
+    Block getBlock(uint64_t blockIndex) const;
+
+    /**
+     * Get a empty block or a new block if the buffer is exhausted.
+     * If the last allocated block size is less than blockSize,
+     * the empty block size is blockSize - lastBlockSize.
+     * Otherwise, the empty block size is blockSize.
+     * @return a empty block object
+     */
+    Block getEmptyBlock();
+
+    /**
+     * Get the number of allocated blocks

Review Comment:
   Done.



-- 
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: issues-unsubscribe@orc.apache.org

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


[GitHub] [orc] coderex2522 commented on a diff in pull request #1271: ORC-1280:[C++] Implement block-based buffer(Part I)

Posted by GitBox <gi...@apache.org>.
coderex2522 commented on code in PR #1271:
URL: https://github.com/apache/orc/pull/1271#discussion_r990742809


##########
c++/src/BlockBuffer.cc:
##########
@@ -0,0 +1,85 @@
+/**
+ * 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.
+ */
+
+#include "BlockBuffer.hh"
+
+#include <algorithm>
+
+namespace orc {
+
+  BlockBuffer::BlockBuffer(MemoryPool& pool, uint64_t _blockSize)
+      : memoryPool(pool),
+        currentSize(0),
+        currentCapacity(0),
+        blockSize(_blockSize) {
+    if (blockSize == 0) {
+      throw std::logic_error("Block size cannot be zero");
+    }
+    reserve(blockSize);
+  }
+
+  BlockBuffer::~BlockBuffer() {
+    for (size_t i = 0; i < blocks.size(); ++i) {
+      memoryPool.free(blocks[i]);
+    }
+    blocks.clear();
+    currentSize = currentCapacity = 0;
+  }
+
+  BlockBuffer::Block BlockBuffer::getBlock(uint64_t blockIndex) const {
+    if (blockIndex >= getBlockNumber()) {
+      throw std::out_of_range("Block index out of range");
+    }
+    return Block(blocks[blockIndex],
+                 std::min(currentSize - blockIndex * blockSize, blockSize));
+  }
+
+  BlockBuffer::Block BlockBuffer::getNextBlock() {
+    if (currentSize < currentCapacity) {
+      Block emptyBlock(
+          blocks[currentSize / blockSize] + currentSize % blockSize,
+          blockSize - currentSize % blockSize);
+      currentSize = (currentSize / blockSize + 1) * blockSize;

Review Comment:
   If user only uses part of the block, the size of BlockBuffer can currently be set via the resize() function. BlockBuffer class is temporarily used to replace DataBuffer in BufferedOutputStream, and the existing functions conform to BufferedOutputStream's usage behavior.
   
   The append function can be added additionally later if there are usage scenarios.



-- 
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: issues-unsubscribe@orc.apache.org

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