You are viewing a plain text version of this content. The canonical link for it is here.
Posted to commits@doris.apache.org by GitBox <gi...@apache.org> on 2020/02/24 12:16:19 UTC

[GitHub] [incubator-doris] lingbin opened a new pull request #2983: Add block layer to storage-engine

lingbin opened a new pull request #2983: Add block layer to storage-engine
URL: https://github.com/apache/incubator-doris/pull/2983
 
 
   The abstraction of the Block layer, inspired by Kudu, lies between the "business
   layer" and the "underlying file storage layer" (`Env`), making them no longer
   strongly coupled.
   
   In this way, for the business layer (such as `SegmentWriter`),
   there is no need to directly do the file operation, which will bring better
   encapsulation. An ideal situation in the future is: when we need to support a
   new file storage system, we only need to add a corresponding type of
   BlockManager without modifying the business code (such as `SegmentWriter`).
   
   With the Block layer, there are some benefits:
   
   1. First and foremost, the mapping relationship between data and `Env` is more
      flexible. For example, in the storage engine, the data of the tablet can be
      placed in multiple file systems (`Env`) at the same time. That is, one-to-many
      relationships can be supported. For example: one on the local and one on the
      remote storage.
   2. The mapping relationship between blocks and files can be adjusted, for example,
      it may not be a one-to-one relationship. For example, the data of multiple
      blocks can be stored in a physical file, which can reduce the number of files
      that need to be opened during querying. It is like `LogBlockManager` in Kudu.
   3. We can move the opened-file-cache under the Block layer, which can automatically
      close and open the files used by the upper layer, so that the upper business
      level does not need to be aware of the restrictions of the file handle at all
      (This problem is often encountered online now).
   4. Better automatic cleanup logic when there are exceptions. For example, a block
      that is not closed explicitly can automatically clean up its corresponding file,
      thereby avoiding generating most garbage files.
   5. More convenient for batch file creation and deletion. Some business operations
      create multiple files, such as compaction. At present, the processing flow that
      these files go through is executed one by one: 1) creation; 2) writing data;
      3) fsync to disk. But in fact, this is not necessary, we only need to fsync this
      batch of files at the end. The advantage is that it can give the operating system
      more opportunities to perform IO merge, thereby improving performance. However,
      this operation is relatively tedious, there is no need to be coupled in the
      business code, it is an ideal place to put it in the Block layer.
   
   This is the first patch, just add related classes, laying the groundwork for later
   switching of read and write logic.

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman merged pull request #2983: Add block layer to storage-engine

Posted by GitBox <gi...@apache.org>.
morningman merged pull request #2983: Add block layer to storage-engine
URL: https://github.com/apache/incubator-doris/pull/2983
 
 
   

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org


[GitHub] [incubator-doris] morningman commented on a change in pull request #2983: Add block layer to storage-engine

Posted by GitBox <gi...@apache.org>.
morningman commented on a change in pull request #2983: Add block layer to storage-engine
URL: https://github.com/apache/incubator-doris/pull/2983#discussion_r386032512
 
 

 ##########
 File path: be/src/olap/fs/file_block_manager.h
 ##########
 @@ -0,0 +1,119 @@
+// 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.
+
+#pragma once
+
+#include <cstdint>
+#include <memory>
+#include <string>
+#include <unordered_set>
+#include <vector>
+
+#include "common/status.h"
+#include "olap/fs/block_manager.h"
+
+namespace doris {
+
+class BlockId;
+class Env;
+class MemTracker;
+class RandomAccessFile;
+
+namespace fs {
+namespace internal {
+
+class FileReadableBlock;
+class FileWritableBlock;
+struct BlockManagerMetrics;
+
+} // namespace internal
+
+// TODO(lingbin): When we create a batch of blocks(blocks are created one by one),
+// eg, when we do a compaction,  multiple files will be generated in sequence.
+// For this scenario, we should have a mechanism that can give the Operating System
+// more opportunities to perform IO merge.
+
+// A file-backed block storage implementation.
+//
+// This is a naive block implementation which maps each block to its own
+// file on disk.
+//
+// The block manager can take advantage of multiple filesystem paths.
+//
+// When creating blocks, the block manager will place blocks based on the
+// provided CreateBlockOptions.
+
+// The file-backed block manager.
+class FileBlockManager : public BlockManager {
+public:
+    // Note: all objects passed as pointers should remain alive for the lifetime
+    // of the block manager.
+    FileBlockManager(Env* env, BlockManagerOptions opts);
+    virtual ~FileBlockManager();
+
+    Status open() override;
+
+    Status create_block(const CreateBlockOptions& opts,
+                        std::unique_ptr<WritableBlock>* block) override;
+    Status open_block(const std::string& path, std::unique_ptr<ReadableBlock>* block) override;
+
+    Status get_all_block_ids(std::vector<BlockId>* block_ids) override {
+        // TODO(lingbin): to be implemented after we assign each block an id
+        return Status::OK();
+    };
+
+private:
+    friend class internal::FileReadableBlock;
+    friend class internal::FileWritableBlock;
+
+    // Deletes an existing block, allowing its space to be reclaimed by the
+    // filesystem. The change is immediately made durable.
+    //
+    // Blocks may be deleted while they are open for reading or writing;
+    // the actual deletion will take place after the last open reader or
+    // writer is closed.
+    Status _delete_block(const std::string& path);
+
+    // Synchronizes the metadata for a block with the given location.
+    Status _sync_metadata(const std::string& path);
+
+    Env* env() const { return _env; }
 
 Review comment:
   Why there is a private env() method?

----------------------------------------------------------------
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.
 
For queries about this service, please contact Infrastructure at:
users@infra.apache.org


With regards,
Apache Git Services

---------------------------------------------------------------------
To unsubscribe, e-mail: commits-unsubscribe@doris.apache.org
For additional commands, e-mail: commits-help@doris.apache.org