pulsar-commits mailing list archives

Site index · List index
Message view « Date » · « Thread »
Top « Date » · « Thread »
From GitBox <...@apache.org>
Subject [GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test
Date Wed, 02 May 2018 10:43:12 GMT
ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream
implementation and test
URL: https://github.com/apache/incubator-pulsar/pull/1678#discussion_r185453817
 
 

 ##########
 File path: pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##########
 @@ -0,0 +1,219 @@
+/**
+ * Licensed to the Apache Software Foundation (ASF) under one
+ * or more contributor license agreements.  See the NOTICE file
+ * distributed with this work for additional information
+ * regarding copyright ownership.  The ASF licenses this file
+ * to you under the Apache License, Version 2.0 (the
+ * "License"); you may not use this file except in compliance
+ * with the License.  You may obtain a copy of the License at
+ *
+ *   http://www.apache.org/licenses/LICENSE-2.0
+ *
+ * Unless required by applicable law or agreed to in writing,
+ * software distributed under the License is distributed on an
+ * "AS IS" BASIS, WITHOUT WARRANTIES OR CONDITIONS OF ANY
+ * KIND, either express or implied.  See the License for the
+ * specific language governing permissions and limitations
+ * under the License.
+ */
+package org.apache.pulsar.broker.s3offload.impl;
+
+import static com.google.common.base.Preconditions.checkState;
+
+import com.google.common.collect.Lists;
+import com.google.common.primitives.Ints;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.CompositeByteBuf;
+import io.netty.buffer.PooledByteBufAllocator;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.ExecutionException;
+import org.apache.bookkeeper.client.api.LedgerEntries;
+import org.apache.bookkeeper.client.api.LedgerEntry;
+import org.apache.bookkeeper.client.api.ReadHandle;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ *
+ * The BlockAwareSegmentInputStream for each cold storage data block.
+ * It contains a byte buffer, which contains all the content for this data block.
+ *      DataBlockHeader + entries(each with format[[entry_size -- int][entry_id -- long][entry_data]])
+ *
+ */
+public class BlockAwareSegmentInputStream extends InputStream {
+    private static final Logger log = LoggerFactory.getLogger(BlockAwareSegmentInputStream.class);
+
+    private static final byte[] blockEndPadding = Ints.toByteArray(0xFEDCDEAD);
+
+    private final ReadHandle ledger;
+    private final long startEntryId;
+    private final int blockSize;
+
+    // Number of Message entries that read from ledger and been readout from this InputStream.
+    private int blockEntryCount;
+    // Number of payload Bytes read from ledger, and has been has been kept in this InputStream.
+    private int payloadBytesHave;
+    // Number of bytes that has been kept in this InputStream.
+    private int blockBytesHave;
+
+    // tracking read status for both header and entries.
+    // Bytes that already been read from this InputStream
+    private int bytesReadOffset = 0;
+    // Byte from this index is all padding byte
+    private int dataBlockFullOffset;
+    private final InputStream dataBlockHeaderStream;
+
+    // how many entries want to read from ReadHandle each time.
+    private static final int entriesNumberEachRead = 100;
+    // buf the entry size and entry id.
+    private static final int entryHeaderSize = 4 /* entry size*/ + 8 /* entry id */;
+    // Keep a list of all entries ByteBuf, each element contains 2 buf: entry header and
entry content.
+    private List<CompositeByteBuf> entriesByteBuf = null;
+
+    public BlockAwareSegmentInputStream(ReadHandle ledger, long startEntryId, int blockSize)
{
+        this.ledger = ledger;
+        this.startEntryId = startEntryId;
+        this.blockSize = blockSize;
+        this.dataBlockHeaderStream = DataBlockHeaderImpl.of(blockSize, startEntryId).toStream();
+        this.blockBytesHave = DataBlockHeaderImpl.getDataStartOffset();
+        this.payloadBytesHave = 0;
+        this.blockEntryCount = 0;
+        this.dataBlockFullOffset = blockSize;
+        this.entriesByteBuf = Lists.newLinkedList();
+    }
+
+    // read ledger entries.
+    private int readEntries() throws IOException {
+        checkState(bytesReadOffset >= DataBlockHeaderImpl.getDataStartOffset());
+        checkState(bytesReadOffset < dataBlockFullOffset);
+
+        try {
+            // once reach the end of entry buffer, start a new read.
+            if (entriesByteBuf.isEmpty()) {
+                readNextEntriesFromLedger();
+                log.debug("After readNextEntriesFromLedger: bytesReadOffset: {}, blockBytesHave:
{}",
+                        bytesReadOffset, blockBytesHave);
+            }
+
+            // always read from the first ByteBuf in the list, once read all of its content
remove it.
+            ByteBuf entryByteBuf = entriesByteBuf.get(0);
+            int ret = entryByteBuf.readByte();
+            bytesReadOffset ++;
+
+            if (entryByteBuf.readableBytes() == 0) {
+                entryByteBuf.release();
+                entriesByteBuf.remove(0);
+            }
+
+            return ret;
+        } catch (InterruptedException | ExecutionException e) {
+            log.error("Exception when get CompletableFuture<LedgerEntries>. ", e);
+            throw new IOException(e);
+        }
+    }
+
+    // read entries from ledger, and pre-handle the returned ledgerEntries.
+    private void readNextEntriesFromLedger() throws InterruptedException, ExecutionException
{
 
 Review comment:
   readNextEntriesFromLedger is doing too many things:
   1. reads from ledger
   2. updates cursor for where we've read to in ledger
   3. updates number of entries written in this stream
   4. updates number of bytes written in this stream
   5. tracks whether there's enough space left in the block
   6. updates end of block when there is not enough space
   
   It should only do the first 2 as they concern reading. The rest of the actions are for
"writing". In fact, I would make readNextEntriesFromLedger only do 1 and have readEntries
deal with counting the entries.
   
   So rather than having:
   ```
   void readNextEntriesFromLedger()
   ```
   have
   ```
   List<ByteBuf> readNextEntries(long firstEntry, long maxNumberEntries)
   ```
   
   Then in readEntries() increment blockEntryCount when we start reading from a new entry.
   
   ```
   // once reach the end of entry buffer, start a new read.
   if (entriesByteBuf.isEmpty()) {
       long firstEntryToRead = startEntryId + blockEntryCount;
       entriesByteBuf = readNextEntriesFromLedger(firstEntryToRead, ENTRIES_PER_READ);
       log.debug("After readNextEntriesFromLedger: bytesReadOffset: {}, blockBytesHave: {}",
                 bytesReadOffset, blockBytesHave);
   }
   
   // always read from the first ByteBuf in the list, once read all of its content remove
it.
   ByteBuf entryByteBuf = entriesByteBuf.get(0);
   int ret = entryByteBuf.readByte();
   
   bytesReadOffset ++;
   
   if (entryByteBuf.readableBytes() == 0) {
       entryByteBuf.release();
       entriesByteBuf.remove(0);
   
       blockEntryCount++;
   }
   
   return ret;
   ```
   This way block entry count is being modified and used in the same method (i.e. you can
see both uses in a single screen of code).
   
   Methods with no parameters and a void return make code hard to understand as they are,
by their very nature, full of side effects. If you want to understand a method that calls
a zero-parameter void method, then you have to understand the whole zero-parameter void method
also.
   
   Similarly, we could move the handling for the padding into readEntries(), and have it all
in one place.
   
   ```
   // once reach the end of entry buffer, start a new read.
   if (entriesByteBuf.isEmpty()) {
       long firstEntryToRead = startEntryId + blockEntryCount;
       entriesByteBuf = readNextEntriesFromLedger(firstEntryToRead, ENTRIES_PER_READ);
       log.debug("After readNextEntriesFromLedger: bytesReadOffset: {}, blockBytesHave: {}",
                 bytesReadOffset, blockBytesHave);
   }
   
   // always read from the first ByteBuf in the list, once read all of its content remove
it.
   ByteBuf entryByteBuf = entriesByteBuf.get(0);
   if ((bytesReadOffset + entryByteBuf.bytesAvailable()) <= blockSize) {
       int ret = entryByteBuf.readByte();
       bytesReadOffset ++;
   
       if (entryByteBuf.readableBytes() == 0) {
           entryByteBuf.release();
           entriesByteBuf.remove(0);
           blockEntryCount++;
       }
       return ret;
   } else {
       return BLOCK_PADDING[bytesReadOffset++ % 4]; 
   }
   ```
   
   Again, it's mutation and usage in the same screen of text. And it removes the need for
2 member variable2 (dataBlockFullOffset & blockBytesHave).
   

----------------------------------------------------------------
This is an automated message from the Apache Git Service.
To respond to the message, please log on 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

Mime
View raw message