[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185578889
 
 

 ##
 File path: 
pulsar-broker/src/test/java/org/apache/pulsar/s3offload/BlockAwareSegmentInputStreamTest.java
 ##
 @@ -0,0 +1,396 @@
+/**
+ * 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.s3offload;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.fail;
+import static org.testng.internal.junit.ArrayAsserts.assertArrayEquals;
+
+import com.google.common.io.ByteStreams;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.io.ByteArrayInputStream;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.stream.IntStream;
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
+import org.apache.bookkeeper.client.api.LedgerEntries;
+import org.apache.bookkeeper.client.api.LedgerEntry;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+import org.apache.bookkeeper.client.api.ReadHandle;
+import org.apache.pulsar.broker.s3offload.DataBlockHeader;
+import 
org.apache.pulsar.broker.s3offload.impl.BlockAwareSegmentInputStreamImpl;
+import org.apache.pulsar.broker.s3offload.impl.DataBlockHeaderImpl;
+import org.testng.annotations.Test;
+import org.testng.collections.Lists;
+
+@Slf4j
+public class BlockAwareSegmentInputStreamTest {
+@Data
+class MockLedgerEntry implements LedgerEntry {
+public byte blockPadding = 0xB;
+long ledgerId;
+long entryId;
+long length;
+byte entryBytes[];
+ByteBuf entryBuffer;
+
+MockLedgerEntry(long ledgerId, long entryId, long length) {
+this.ledgerId = ledgerId;
+this.entryId = entryId;
+this.length = length;
+this.entryBytes = new byte[(int)length];
+entryBuffer = Unpooled.wrappedBuffer(entryBytes);
+entryBuffer.writerIndex(0);
+IntStream.range(0, (int)length).forEach(i -> 
entryBuffer.writeByte(blockPadding));
+}
+
+@Override
+public ByteBuffer getEntryNioBuffer() {
+return null;
+}
+
+@Override
+public LedgerEntry duplicate() {
+return null;
+}
+
+@Override
+public void close() {
+entryBuffer.release();
+}
+}
+
+@Data
+class MockLedgerEntries implements LedgerEntries {
+int ledgerId;
+int startEntryId;
+int count;
+int entrySize;
+List entries;
+
+MockLedgerEntries(int ledgerId, int startEntryId, int count, int 
entrySize) {
+this.ledgerId = ledgerId;
+this.startEntryId = startEntryId;
+this.count = count;
+this.entrySize = entrySize;
+this.entries = Lists.newArrayList(count);
+
+IntStream.range(startEntryId, startEntryId + count).forEach(i ->
+entries.add(new MockLedgerEntry(ledgerId, i, entrySize)));
+}
+
+@Override
+public void close() {
+entries.clear();
+}
+
+@Override
+public LedgerEntry getEntry(long entryId) {
+if (entryId < startEntryId || entryId >= startEntryId + count) {
+return null;
+}
+
+return entries.get(((int)entryId - startEntryId));
+}
+
+@Override
+public Iterator iterator() {
+return entries.iterator();
+}
+}
+
+class MockReadHandle implements ReadHandle {
+int ledgerId;
+int entrySize;
+int lac;
+MockReadHandle(int ledgerId, int entrySize, int lac) {
+this.ledgerId = ledgerId;
+this.entrySize = entrySize;
+this.lac = lac;
+  

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185564229
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStreamImpl.java
 ##
 @@ -0,0 +1,217 @@
+/**
+ * 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.apache.pulsar.broker.s3offload.BlockAwareSegmentInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The BlockAwareSegmentInputStreamImpl for each cold storage data block.
+ * It gets data from ledger, and will be read out the content for a data block.
+ * DataBlockHeader + entries(each with format[[entry_size -- int][entry_id -- 
long][entry_data]]) + padding
+ */
+public class BlockAwareSegmentInputStreamImpl extends InputStream implements 
BlockAwareSegmentInputStream {
+private static final Logger log = 
LoggerFactory.getLogger(BlockAwareSegmentInputStreamImpl.class);
+
+private static final byte[] BLOCK_END_PADDING = 
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;
+
+// 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 ENTRIES_PER_READ = 100;
+// buf the entry size and entry id.
+private static final int ENTRY_HEADER_SIZE = 4 /* entry size*/ + 8 /* 
entry id */;
+// Keep a list of all entries ByteBuf, each ByteBuf contains 2 buf: entry 
header and entry content.
+private List entriesByteBuf = null;
+
+public BlockAwareSegmentInputStreamImpl(ReadHandle ledger, long 
startEntryId, int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeaderStream = DataBlockHeaderImpl.of(blockSize, 
startEntryId).toStream();
+this.blockEntryCount = 0;
+this.dataBlockFullOffset = blockSize;
+this.entriesByteBuf = Lists.newLinkedList();
+}
+
+// read ledger entries.
+private int readEntries() throws IOException {
+checkState(bytesReadOffset >= 
DataBlockHeaderImpl.getDataStartOffset());
+checkState(bytesReadOffset < blockSize);
+
+// once reach the end of entry buffer, start a new read.
+if (bytesReadOffset < dataBlockFullOffset && entriesByteBuf.isEmpty()) 
{
 
 Review comment:
   it could be simplifier more if entriesByteBuf cleanup is left until close.
   
   ```
if (entriesByteBuf.isEmpty()) {
entriesByteBuf = readNextEntriesFromLedger(startEntryId + 
blockEntryCount, ENTRIES_PER_READ);
   }
   if ((!entriesByteBuf.isEmpty()) && bytesReadOffset + 
entriesByteBuf.get(0).readableBytes() > blockSize) {
  // not able to place a new Entry.
  dataBlockFullOffset = bytesReadOffset;
   
  return BLOCK_END_PADDING[(bytesReadOffset++ - dataBlockFullOffset) % 4];
   } else {
   ...
   ```


This is an automated message from the Apache Git 

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185561252
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStreamImpl.java
 ##
 @@ -0,0 +1,217 @@
+/**
+ * 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.apache.pulsar.broker.s3offload.BlockAwareSegmentInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The BlockAwareSegmentInputStreamImpl for each cold storage data block.
+ * It gets data from ledger, and will be read out the content for a data block.
+ * DataBlockHeader + entries(each with format[[entry_size -- int][entry_id -- 
long][entry_data]]) + padding
+ */
+public class BlockAwareSegmentInputStreamImpl extends InputStream implements 
BlockAwareSegmentInputStream {
+private static final Logger log = 
LoggerFactory.getLogger(BlockAwareSegmentInputStreamImpl.class);
+
+private static final byte[] BLOCK_END_PADDING = 
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;
+
+// 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 ENTRIES_PER_READ = 100;
+// buf the entry size and entry id.
+private static final int ENTRY_HEADER_SIZE = 4 /* entry size*/ + 8 /* 
entry id */;
+// Keep a list of all entries ByteBuf, each ByteBuf contains 2 buf: entry 
header and entry content.
+private List entriesByteBuf = null;
+
+public BlockAwareSegmentInputStreamImpl(ReadHandle ledger, long 
startEntryId, int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeaderStream = DataBlockHeaderImpl.of(blockSize, 
startEntryId).toStream();
+this.blockEntryCount = 0;
+this.dataBlockFullOffset = blockSize;
+this.entriesByteBuf = Lists.newLinkedList();
+}
+
+// read ledger entries.
+private int readEntries() throws IOException {
+checkState(bytesReadOffset >= 
DataBlockHeaderImpl.getDataStartOffset());
+checkState(bytesReadOffset < blockSize);
+
+// once reach the end of entry buffer, start a new read.
+if (bytesReadOffset < dataBlockFullOffset && entriesByteBuf.isEmpty()) 
{
+entriesByteBuf = readNextEntriesFromLedger(startEntryId + 
blockEntryCount, ENTRIES_PER_READ);
+if ((!entriesByteBuf.isEmpty()) && bytesReadOffset + 
entriesByteBuf.get(0).readableBytes() > blockSize) {
+// not able to place a new Entry.
+entriesByteBuf.forEach(buf -> buf.release());
+entriesByteBuf.clear();
+dataBlockFullOffset = bytesReadOffset;
+}
+}
+
+if (bytesReadOffset < dataBlockFullOffset) {
+// always read from the first ByteBuf in the list, once read all 
of its content remove it.
+ByteBuf entryByteBuf = 

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185566416
 
 

 ##
 File path: 
pulsar-broker/src/test/java/org/apache/pulsar/s3offload/BlockAwareSegmentInputStreamTest.java
 ##
 @@ -0,0 +1,396 @@
+/**
+ * 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.s3offload;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.fail;
+import static org.testng.internal.junit.ArrayAsserts.assertArrayEquals;
+
+import com.google.common.io.ByteStreams;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.io.ByteArrayInputStream;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.stream.IntStream;
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
+import org.apache.bookkeeper.client.api.LedgerEntries;
+import org.apache.bookkeeper.client.api.LedgerEntry;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+import org.apache.bookkeeper.client.api.ReadHandle;
+import org.apache.pulsar.broker.s3offload.DataBlockHeader;
+import 
org.apache.pulsar.broker.s3offload.impl.BlockAwareSegmentInputStreamImpl;
+import org.apache.pulsar.broker.s3offload.impl.DataBlockHeaderImpl;
+import org.testng.annotations.Test;
+import org.testng.collections.Lists;
+
+@Slf4j
+public class BlockAwareSegmentInputStreamTest {
+@Data
+class MockLedgerEntry implements LedgerEntry {
+public byte blockPadding = 0xB;
+long ledgerId;
+long entryId;
+long length;
+byte entryBytes[];
+ByteBuf entryBuffer;
+
+MockLedgerEntry(long ledgerId, long entryId, long length) {
+this.ledgerId = ledgerId;
+this.entryId = entryId;
+this.length = length;
+this.entryBytes = new byte[(int)length];
+entryBuffer = Unpooled.wrappedBuffer(entryBytes);
+entryBuffer.writerIndex(0);
+IntStream.range(0, (int)length).forEach(i -> 
entryBuffer.writeByte(blockPadding));
+}
+
+@Override
+public ByteBuffer getEntryNioBuffer() {
+return null;
+}
+
+@Override
+public LedgerEntry duplicate() {
+return null;
+}
+
+@Override
+public void close() {
+entryBuffer.release();
+}
+}
+
+@Data
+class MockLedgerEntries implements LedgerEntries {
+int ledgerId;
+int startEntryId;
+int count;
+int entrySize;
+List entries;
+
+MockLedgerEntries(int ledgerId, int startEntryId, int count, int 
entrySize) {
+this.ledgerId = ledgerId;
+this.startEntryId = startEntryId;
+this.count = count;
+this.entrySize = entrySize;
+this.entries = Lists.newArrayList(count);
+
+IntStream.range(startEntryId, startEntryId + count).forEach(i ->
+entries.add(new MockLedgerEntry(ledgerId, i, entrySize)));
+}
+
+@Override
+public void close() {
+entries.clear();
+}
+
+@Override
+public LedgerEntry getEntry(long entryId) {
+if (entryId < startEntryId || entryId >= startEntryId + count) {
+return null;
+}
+
+return entries.get(((int)entryId - startEntryId));
+}
+
+@Override
+public Iterator iterator() {
+return entries.iterator();
+}
+}
+
+class MockReadHandle implements ReadHandle {
+int ledgerId;
+int entrySize;
+int lac;
+MockReadHandle(int ledgerId, int entrySize, int lac) {
+this.ledgerId = ledgerId;
+this.entrySize = entrySize;
+this.lac = lac;
+  

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185568883
 
 

 ##
 File path: 
pulsar-broker/src/test/java/org/apache/pulsar/s3offload/BlockAwareSegmentInputStreamTest.java
 ##
 @@ -0,0 +1,396 @@
+/**
+ * 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.s3offload;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.fail;
+import static org.testng.internal.junit.ArrayAsserts.assertArrayEquals;
+
+import com.google.common.io.ByteStreams;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.io.ByteArrayInputStream;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.stream.IntStream;
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
+import org.apache.bookkeeper.client.api.LedgerEntries;
+import org.apache.bookkeeper.client.api.LedgerEntry;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+import org.apache.bookkeeper.client.api.ReadHandle;
+import org.apache.pulsar.broker.s3offload.DataBlockHeader;
+import 
org.apache.pulsar.broker.s3offload.impl.BlockAwareSegmentInputStreamImpl;
+import org.apache.pulsar.broker.s3offload.impl.DataBlockHeaderImpl;
+import org.testng.annotations.Test;
+import org.testng.collections.Lists;
+
+@Slf4j
+public class BlockAwareSegmentInputStreamTest {
+@Data
+class MockLedgerEntry implements LedgerEntry {
+public byte blockPadding = 0xB;
+long ledgerId;
+long entryId;
+long length;
+byte entryBytes[];
+ByteBuf entryBuffer;
+
+MockLedgerEntry(long ledgerId, long entryId, long length) {
+this.ledgerId = ledgerId;
+this.entryId = entryId;
+this.length = length;
+this.entryBytes = new byte[(int)length];
+entryBuffer = Unpooled.wrappedBuffer(entryBytes);
+entryBuffer.writerIndex(0);
+IntStream.range(0, (int)length).forEach(i -> 
entryBuffer.writeByte(blockPadding));
+}
+
+@Override
+public ByteBuffer getEntryNioBuffer() {
+return null;
+}
+
+@Override
+public LedgerEntry duplicate() {
+return null;
+}
+
+@Override
+public void close() {
+entryBuffer.release();
+}
+}
+
+@Data
+class MockLedgerEntries implements LedgerEntries {
+int ledgerId;
+int startEntryId;
+int count;
+int entrySize;
+List entries;
+
+MockLedgerEntries(int ledgerId, int startEntryId, int count, int 
entrySize) {
+this.ledgerId = ledgerId;
+this.startEntryId = startEntryId;
+this.count = count;
+this.entrySize = entrySize;
+this.entries = Lists.newArrayList(count);
+
+IntStream.range(startEntryId, startEntryId + count).forEach(i ->
+entries.add(new MockLedgerEntry(ledgerId, i, entrySize)));
+}
+
+@Override
+public void close() {
+entries.clear();
+}
+
+@Override
+public LedgerEntry getEntry(long entryId) {
+if (entryId < startEntryId || entryId >= startEntryId + count) {
+return null;
+}
+
+return entries.get(((int)entryId - startEntryId));
+}
+
+@Override
+public Iterator iterator() {
+return entries.iterator();
+}
+}
+
+class MockReadHandle implements ReadHandle {
+int ledgerId;
+int entrySize;
+int lac;
+MockReadHandle(int ledgerId, int entrySize, int lac) {
+this.ledgerId = ledgerId;
+this.entrySize = entrySize;
+this.lac = lac;
+  

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185560570
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStreamImpl.java
 ##
 @@ -0,0 +1,217 @@
+/**
+ * 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.apache.pulsar.broker.s3offload.BlockAwareSegmentInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The BlockAwareSegmentInputStreamImpl for each cold storage data block.
+ * It gets data from ledger, and will be read out the content for a data block.
+ * DataBlockHeader + entries(each with format[[entry_size -- int][entry_id -- 
long][entry_data]]) + padding
+ */
+public class BlockAwareSegmentInputStreamImpl extends InputStream implements 
BlockAwareSegmentInputStream {
+private static final Logger log = 
LoggerFactory.getLogger(BlockAwareSegmentInputStreamImpl.class);
+
+private static final byte[] BLOCK_END_PADDING = 
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;
+
+// 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 ENTRIES_PER_READ = 100;
+// buf the entry size and entry id.
+private static final int ENTRY_HEADER_SIZE = 4 /* entry size*/ + 8 /* 
entry id */;
+// Keep a list of all entries ByteBuf, each ByteBuf contains 2 buf: entry 
header and entry content.
+private List entriesByteBuf = null;
+
+public BlockAwareSegmentInputStreamImpl(ReadHandle ledger, long 
startEntryId, int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeaderStream = DataBlockHeaderImpl.of(blockSize, 
startEntryId).toStream();
+this.blockEntryCount = 0;
+this.dataBlockFullOffset = blockSize;
+this.entriesByteBuf = Lists.newLinkedList();
+}
+
+// read ledger entries.
+private int readEntries() throws IOException {
+checkState(bytesReadOffset >= 
DataBlockHeaderImpl.getDataStartOffset());
+checkState(bytesReadOffset < blockSize);
+
+// once reach the end of entry buffer, start a new read.
+if (bytesReadOffset < dataBlockFullOffset && entriesByteBuf.isEmpty()) 
{
+entriesByteBuf = readNextEntriesFromLedger(startEntryId + 
blockEntryCount, ENTRIES_PER_READ);
+if ((!entriesByteBuf.isEmpty()) && bytesReadOffset + 
entriesByteBuf.get(0).readableBytes() > blockSize) {
+// not able to place a new Entry.
+entriesByteBuf.forEach(buf -> buf.release());
+entriesByteBuf.clear();
+dataBlockFullOffset = bytesReadOffset;
+}
+}
+
+if (bytesReadOffset < dataBlockFullOffset) {
+// always read from the first ByteBuf in the list, once read all 
of its content remove it.
+ByteBuf entryByteBuf = 

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185557470
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStreamImpl.java
 ##
 @@ -0,0 +1,217 @@
+/**
+ * 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.apache.pulsar.broker.s3offload.BlockAwareSegmentInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The BlockAwareSegmentInputStreamImpl for each cold storage data block.
+ * It gets data from ledger, and will be read out the content for a data block.
+ * DataBlockHeader + entries(each with format[[entry_size -- int][entry_id -- 
long][entry_data]]) + padding
+ */
+public class BlockAwareSegmentInputStreamImpl extends InputStream implements 
BlockAwareSegmentInputStream {
+private static final Logger log = 
LoggerFactory.getLogger(BlockAwareSegmentInputStreamImpl.class);
+
+private static final byte[] BLOCK_END_PADDING = 
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;
+
+// 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 ENTRIES_PER_READ = 100;
+// buf the entry size and entry id.
+private static final int ENTRY_HEADER_SIZE = 4 /* entry size*/ + 8 /* 
entry id */;
+// Keep a list of all entries ByteBuf, each ByteBuf contains 2 buf: entry 
header and entry content.
+private List entriesByteBuf = null;
+
+public BlockAwareSegmentInputStreamImpl(ReadHandle ledger, long 
startEntryId, int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeaderStream = DataBlockHeaderImpl.of(blockSize, 
startEntryId).toStream();
+this.blockEntryCount = 0;
+this.dataBlockFullOffset = blockSize;
+this.entriesByteBuf = Lists.newLinkedList();
+}
+
+// read ledger entries.
+private int readEntries() throws IOException {
+checkState(bytesReadOffset >= 
DataBlockHeaderImpl.getDataStartOffset());
+checkState(bytesReadOffset < blockSize);
+
+// once reach the end of entry buffer, start a new read.
+if (bytesReadOffset < dataBlockFullOffset && entriesByteBuf.isEmpty()) 
{
+entriesByteBuf = readNextEntriesFromLedger(startEntryId + 
blockEntryCount, ENTRIES_PER_READ);
+if ((!entriesByteBuf.isEmpty()) && bytesReadOffset + 
entriesByteBuf.get(0).readableBytes() > blockSize) {
+// not able to place a new Entry.
+entriesByteBuf.forEach(buf -> buf.release());
+entriesByteBuf.clear();
+dataBlockFullOffset = bytesReadOffset;
+}
+}
+
+if (bytesReadOffset < dataBlockFullOffset) {
+// always read from the first ByteBuf in the list, once read all 
of its content remove it.
+ByteBuf entryByteBuf = 

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185562881
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStreamImpl.java
 ##
 @@ -0,0 +1,217 @@
+/**
+ * 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.apache.pulsar.broker.s3offload.BlockAwareSegmentInputStream;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+/**
+ * The BlockAwareSegmentInputStreamImpl for each cold storage data block.
+ * It gets data from ledger, and will be read out the content for a data block.
+ * DataBlockHeader + entries(each with format[[entry_size -- int][entry_id -- 
long][entry_data]]) + padding
+ */
+public class BlockAwareSegmentInputStreamImpl extends InputStream implements 
BlockAwareSegmentInputStream {
+private static final Logger log = 
LoggerFactory.getLogger(BlockAwareSegmentInputStreamImpl.class);
+
+private static final byte[] BLOCK_END_PADDING = 
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;
+
+// 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 ENTRIES_PER_READ = 100;
+// buf the entry size and entry id.
+private static final int ENTRY_HEADER_SIZE = 4 /* entry size*/ + 8 /* 
entry id */;
+// Keep a list of all entries ByteBuf, each ByteBuf contains 2 buf: entry 
header and entry content.
+private List entriesByteBuf = null;
+
+public BlockAwareSegmentInputStreamImpl(ReadHandle ledger, long 
startEntryId, int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeaderStream = DataBlockHeaderImpl.of(blockSize, 
startEntryId).toStream();
+this.blockEntryCount = 0;
+this.dataBlockFullOffset = blockSize;
+this.entriesByteBuf = Lists.newLinkedList();
+}
+
+// read ledger entries.
+private int readEntries() throws IOException {
+checkState(bytesReadOffset >= 
DataBlockHeaderImpl.getDataStartOffset());
+checkState(bytesReadOffset < blockSize);
+
+// once reach the end of entry buffer, start a new read.
+if (bytesReadOffset < dataBlockFullOffset && entriesByteBuf.isEmpty()) 
{
 
 Review comment:
   moving the inner check out of the outer check will remove code duplication
   
   ```
if (bytesReadOffset < dataBlockFullOffset && entriesByteBuf.isEmpty()) {
entriesByteBuf = readNextEntriesFromLedger(startEntryId + 
blockEntryCount, ENTRIES_PER_READ);
   }
if ((!entriesByteBuf.isEmpty()) && bytesReadOffset + 
entriesByteBuf.get(0).readableBytes() > blockSize) {
   // not able to place a new Entry.
  entriesByteBuf.forEach(buf -> buf.release());
  entriesByteBuf.clear();
  dataBlockFullOffset = bytesReadOffset;
   
  return BLOCK_END_PADDING[(bytesReadOffset++ - dataBlockFullOffset) % 4];
   } else if 

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185565049
 
 

 ##
 File path: 
pulsar-broker/src/test/java/org/apache/pulsar/s3offload/BlockAwareSegmentInputStreamTest.java
 ##
 @@ -0,0 +1,396 @@
+/**
+ * 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.s3offload;
+
+import static org.testng.Assert.assertEquals;
+import static org.testng.Assert.fail;
+import static org.testng.internal.junit.ArrayAsserts.assertArrayEquals;
+
+import com.google.common.io.ByteStreams;
+import com.google.common.primitives.Ints;
+import com.google.common.primitives.Longs;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.Unpooled;
+import io.netty.util.concurrent.DefaultThreadFactory;
+import java.io.ByteArrayInputStream;
+import java.nio.ByteBuffer;
+import java.util.Iterator;
+import java.util.List;
+import java.util.concurrent.CompletableFuture;
+import java.util.concurrent.Executors;
+import java.util.stream.IntStream;
+import lombok.Data;
+import lombok.extern.slf4j.Slf4j;
+import org.apache.bookkeeper.client.api.LastConfirmedAndEntry;
+import org.apache.bookkeeper.client.api.LedgerEntries;
+import org.apache.bookkeeper.client.api.LedgerEntry;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+import org.apache.bookkeeper.client.api.ReadHandle;
+import org.apache.pulsar.broker.s3offload.DataBlockHeader;
+import 
org.apache.pulsar.broker.s3offload.impl.BlockAwareSegmentInputStreamImpl;
+import org.apache.pulsar.broker.s3offload.impl.DataBlockHeaderImpl;
+import org.testng.annotations.Test;
+import org.testng.collections.Lists;
+
+@Slf4j
+public class BlockAwareSegmentInputStreamTest {
+@Data
+class MockLedgerEntry implements LedgerEntry {
+public byte blockPadding = 0xB;
+long ledgerId;
+long entryId;
+long length;
+byte entryBytes[];
+ByteBuf entryBuffer;
+
+MockLedgerEntry(long ledgerId, long entryId, long length) {
+this.ledgerId = ledgerId;
+this.entryId = entryId;
+this.length = length;
+this.entryBytes = new byte[(int)length];
+entryBuffer = Unpooled.wrappedBuffer(entryBytes);
+entryBuffer.writerIndex(0);
+IntStream.range(0, (int)length).forEach(i -> 
entryBuffer.writeByte(blockPadding));
+}
+
+@Override
+public ByteBuffer getEntryNioBuffer() {
+return null;
+}
+
+@Override
+public LedgerEntry duplicate() {
+return null;
+}
+
+@Override
+public void close() {
+entryBuffer.release();
+}
+}
+
+@Data
+class MockLedgerEntries implements LedgerEntries {
+int ledgerId;
+int startEntryId;
+int count;
+int entrySize;
+List entries;
+
+MockLedgerEntries(int ledgerId, int startEntryId, int count, int 
entrySize) {
+this.ledgerId = ledgerId;
+this.startEntryId = startEntryId;
+this.count = count;
+this.entrySize = entrySize;
+this.entries = Lists.newArrayList(count);
+
+IntStream.range(startEntryId, startEntryId + count).forEach(i ->
+entries.add(new MockLedgerEntry(ledgerId, i, entrySize)));
+}
+
+@Override
+public void close() {
+entries.clear();
+}
+
+@Override
+public LedgerEntry getEntry(long entryId) {
+if (entryId < startEntryId || entryId >= startEntryId + count) {
+return null;
+}
+
+return entries.get(((int)entryId - startEntryId));
+}
+
+@Override
+public Iterator iterator() {
+return entries.iterator();
+}
+}
+
+class MockReadHandle implements ReadHandle {
+int ledgerId;
+int entrySize;
+int lac;
+MockReadHandle(int ledgerId, int entrySize, int lac) {
+this.ledgerId = ledgerId;
+this.entrySize = entrySize;
+this.lac = lac;
+  

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185498765
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##
 @@ -90,68 +84,60 @@ 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 ++;
+// once reach the end of entry buffer, start a new read.
+if (entriesByteBuf.isEmpty()) {
+entriesByteBuf = readNextEntriesFromLedger(startEntryId + 
blockEntryCount, ENTRIES_PER_READ);
 
 Review comment:
   what happens if readNextEntriesFromLedger cannot read any more entries?


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185498146
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##
 @@ -90,68 +84,60 @@ 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 ++;
+// once reach the end of entry buffer, start a new read.
+if (entriesByteBuf.isEmpty()) {
+entriesByteBuf = readNextEntriesFromLedger(startEntryId + 
blockEntryCount, ENTRIES_PER_READ);
+}
 
-if (entryByteBuf.readableBytes() == 0) {
-entryByteBuf.release();
-entriesByteBuf.remove(0);
+// 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++;
+if ((!entriesByteBuf.isEmpty()) && bytesReadOffset + 
entriesByteBuf.get(0).readableBytes() > blockSize) {
+// not able to place a new Entry.
+entriesByteBuf.forEach(buf -> buf.release());
+entriesByteBuf.clear();
+dataBlockFullOffset = bytesReadOffset;
 }
-
-return ret;
-} catch (InterruptedException | ExecutionException e) {
-log.error("Exception when get CompletableFuture. ", 
e);
-throw new IOException(e);
 }
-}
 
-// read entries from ledger, and pre-handle the returned ledgerEntries.
-private void readNextEntriesFromLedger() throws InterruptedException, 
ExecutionException {
-checkState(bytesReadOffset == blockBytesHave);
+return ret;
+}
 
-long start = startEntryId + blockEntryCount;
-long end = Math.min(start + entriesNumberEachRead - 1, 
ledger.getLastAddConfirmed());
+private List readNextEntriesFromLedger(long start, long 
maxNumberEntries) throws IOException {
+long end = Math.min(start + maxNumberEntries - 1, 
ledger.getLastAddConfirmed());
 try (LedgerEntries ledgerEntriesOnce = ledger.readAsync(start, 
end).get()) {
 log.debug("read ledger entries. start: {}, end: {}", start, end);
 
+List entries = Lists.newLinkedList();
+
 Iterator iterator = ledgerEntriesOnce.iterator();
-long entryId = start;
 while (iterator.hasNext()) {
 LedgerEntry entry = iterator.next();
 int entryLength = (int) entry.getLength();
-entryId = entry.getEntryId();
-
-if (blockSize - blockBytesHave >= entryLength + 
entryHeaderSize) {
-// data block has space for this entry, keep this entry
-CompositeByteBuf entryBuf = 
PooledByteBufAllocator.DEFAULT.compositeBuffer();
-ByteBuf entryHeaderBuf = 
PooledByteBufAllocator.DEFAULT.buffer(entryHeaderSize, entryHeaderSize);
-
-entryHeaderBuf.writeInt(entryLength).writeLong(entryId);
-entryBuf.addComponents(entryHeaderBuf, 
entry.getEntryBuffer().retain());
-entryBuf.writerIndex(entryHeaderSize + entryLength);
-
-entriesByteBuf.add(entryBuf);
-
-// set counters
-blockEntryCount++;
-payloadBytesHave += entryLength;
-blockBytesHave += entryLength + entryHeaderSize;
-} else {
-// data block has no space left for a whole message entry
-dataBlockFullOffset = blockBytesHave;
-break;
-}
+long entryId = entry.getEntryId();
+
+CompositeByteBuf entryBuf = 
PooledByteBufAllocator.DEFAULT.compositeBuffer();
+ByteBuf entryHeaderBuf = 
PooledByteBufAllocator.DEFAULT.buffer(ENTRY_HEADER_SIZE, ENTRY_HEADER_SIZE);
+
+

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185496454
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##
 @@ -90,68 +84,60 @@ 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 ++;
+// once reach the end of entry buffer, start a new read.
+if (entriesByteBuf.isEmpty()) {
+entriesByteBuf = readNextEntriesFromLedger(startEntryId + 
blockEntryCount, ENTRIES_PER_READ);
+}
 
-if (entryByteBuf.readableBytes() == 0) {
-entryByteBuf.release();
-entriesByteBuf.remove(0);
+// 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++;
+if ((!entriesByteBuf.isEmpty()) && bytesReadOffset + 
entriesByteBuf.get(0).readableBytes() > blockSize) {
+// not able to place a new Entry.
+entriesByteBuf.forEach(buf -> buf.release());
+entriesByteBuf.clear();
+dataBlockFullOffset = bytesReadOffset;
 }
-
-return ret;
-} catch (InterruptedException | ExecutionException e) {
-log.error("Exception when get CompletableFuture. ", 
e);
-throw new IOException(e);
 }
-}
 
-// read entries from ledger, and pre-handle the returned ledgerEntries.
-private void readNextEntriesFromLedger() throws InterruptedException, 
ExecutionException {
-checkState(bytesReadOffset == blockBytesHave);
+return ret;
+}
 
-long start = startEntryId + blockEntryCount;
-long end = Math.min(start + entriesNumberEachRead - 1, 
ledger.getLastAddConfirmed());
+private List readNextEntriesFromLedger(long start, long 
maxNumberEntries) throws IOException {
+long end = Math.min(start + maxNumberEntries - 1, 
ledger.getLastAddConfirmed());
 try (LedgerEntries ledgerEntriesOnce = ledger.readAsync(start, 
end).get()) {
 log.debug("read ledger entries. start: {}, end: {}", start, end);
 
+List entries = Lists.newLinkedList();
+
 Iterator iterator = ledgerEntriesOnce.iterator();
-long entryId = start;
 while (iterator.hasNext()) {
 LedgerEntry entry = iterator.next();
 int entryLength = (int) entry.getLength();
-entryId = entry.getEntryId();
-
-if (blockSize - blockBytesHave >= entryLength + 
entryHeaderSize) {
-// data block has space for this entry, keep this entry
-CompositeByteBuf entryBuf = 
PooledByteBufAllocator.DEFAULT.compositeBuffer();
-ByteBuf entryHeaderBuf = 
PooledByteBufAllocator.DEFAULT.buffer(entryHeaderSize, entryHeaderSize);
-
-entryHeaderBuf.writeInt(entryLength).writeLong(entryId);
-entryBuf.addComponents(entryHeaderBuf, 
entry.getEntryBuffer().retain());
-entryBuf.writerIndex(entryHeaderSize + entryLength);
-
-entriesByteBuf.add(entryBuf);
-
-// set counters
-blockEntryCount++;
-payloadBytesHave += entryLength;
-blockBytesHave += entryLength + entryHeaderSize;
-} else {
-// data block has no space left for a whole message entry
-dataBlockFullOffset = blockBytesHave;
-break;
-}
+long entryId = entry.getEntryId();
+
+CompositeByteBuf entryBuf = 
PooledByteBufAllocator.DEFAULT.compositeBuffer();
 
 Review comment:
   nit: specify that the compositeBuffer will have 2 buffers since, we know it 
will.


[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185495929
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##
 @@ -90,68 +84,60 @@ 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 ++;
+// once reach the end of entry buffer, start a new read.
+if (entriesByteBuf.isEmpty()) {
+entriesByteBuf = readNextEntriesFromLedger(startEntryId + 
blockEntryCount, ENTRIES_PER_READ);
+}
 
-if (entryByteBuf.readableBytes() == 0) {
-entryByteBuf.release();
-entriesByteBuf.remove(0);
+// 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++;
+if ((!entriesByteBuf.isEmpty()) && bytesReadOffset + 
entriesByteBuf.get(0).readableBytes() > blockSize) {
 
 Review comment:
   What entriesByteBuf is empty, but the first entry in the next read block 
will put you over the blockSize?
   This check needs to move to before line 92 (that's why I had the if-else in 
previous comment).


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185438219
 
 

 ##
 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 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 ++;
+
+ 

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185441127
 
 

 ##
 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);
 
 Review comment:
   BLOCK_PADDING


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185456144
 
 

 ##
 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 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 ++;
+
+ 

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185456500
 
 

 ##
 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 {
 
 Review comment:
   Could we define an interface and impl for this, so it's clear what extra 
data BlockAwareSegmentInputStream exposes?


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185441005
 
 

 ##
 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 */;
 
 Review comment:
   ENTRY_HEADER_SIZE


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185441047
 
 

 ##
 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;
 
 Review comment:
   ENTRIES_PER_READ


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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 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 ++;
+
+ 

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185435514
 
 

 ##
 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 entriesByteBuf = null;
 
 Review comment:
   It can be a List. The user of the buffer doesn't need to know 
what's backing it.


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-05-02 Thread GitBox
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_r185456251
 
 

 ##
 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 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 ++;
+
+ 

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-04-30 Thread GitBox
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_r184955941
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##
 @@ -0,0 +1,293 @@
+/**
+ * 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.Unpooled;
+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.apache.pulsar.broker.s3offload.DataBlockHeader;
+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 DataBlockHeader dataBlockHeader;
+private static int dataBlockHeaderLength = 
DataBlockHeaderImpl.getDataStartOffset();
+private InputStream dataBlockHeaderBuf;
+
+// 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 */;
+// the start offset in data block for this time read from ReadHandle.
+private int entryStartBlockOffset;
+// the end offset in data block for this time read from ReadHandle.
+private int entryEndBlockOffset;
+private byte[] entriesHead = new byte[entriesNumberEachRead * 
entryHeaderSize];
+private ByteBuf entriesHeadBuf = null;
+// Keep a list of all the entries ByteBuf to avoid a memory copy
+private List entriesByteBuf = null;
+private ByteBuf entryByteBuf = null;
+
+public BlockAwareSegmentInputStream(ReadHandle ledger, long startEntryId, 
int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeader = DataBlockHeaderImpl.of(blockSize, startEntryId);
+this.payloadBytesHave = 0;
+this.blockEntryCount = 0;
+this.dataBlockFullOffset = blockSize;
+this.entryStartBlockOffset = DataBlockHeaderImpl.getDataStartOffset();
+this.entryEndBlockOffset = entryStartBlockOffset;
+this.entriesByteBuf = 
Lists.newArrayListWithExpectedSize(entriesNumberEachRead);
+}
+
+// read ledger entries.
+private int readEntries() throws IOException {
+checkState(bytesReadOffset >= dataBlockHeaderLength);
+checkState(bytesReadOffset < dataBlockFullOffset);
+checkState(ledger != null);
+
+  

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-04-30 Thread GitBox
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_r184951858
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##
 @@ -0,0 +1,293 @@
+/**
+ * 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.Unpooled;
+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.apache.pulsar.broker.s3offload.DataBlockHeader;
+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 DataBlockHeader dataBlockHeader;
+private static int dataBlockHeaderLength = 
DataBlockHeaderImpl.getDataStartOffset();
+private InputStream dataBlockHeaderBuf;
+
+// 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 */;
+// the start offset in data block for this time read from ReadHandle.
+private int entryStartBlockOffset;
+// the end offset in data block for this time read from ReadHandle.
+private int entryEndBlockOffset;
+private byte[] entriesHead = new byte[entriesNumberEachRead * 
entryHeaderSize];
+private ByteBuf entriesHeadBuf = null;
+// Keep a list of all the entries ByteBuf to avoid a memory copy
+private List entriesByteBuf = null;
+private ByteBuf entryByteBuf = null;
+
+public BlockAwareSegmentInputStream(ReadHandle ledger, long startEntryId, 
int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeader = DataBlockHeaderImpl.of(blockSize, startEntryId);
+this.payloadBytesHave = 0;
+this.blockEntryCount = 0;
+this.dataBlockFullOffset = blockSize;
+this.entryStartBlockOffset = DataBlockHeaderImpl.getDataStartOffset();
+this.entryEndBlockOffset = entryStartBlockOffset;
+this.entriesByteBuf = 
Lists.newArrayListWithExpectedSize(entriesNumberEachRead);
+}
+
+// read ledger entries.
+private int readEntries() throws IOException {
+checkState(bytesReadOffset >= dataBlockHeaderLength);
+checkState(bytesReadOffset < dataBlockFullOffset);
+checkState(ledger != null);
+
+  

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-04-30 Thread GitBox
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_r184959173
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##
 @@ -0,0 +1,293 @@
+/**
+ * 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.Unpooled;
+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.apache.pulsar.broker.s3offload.DataBlockHeader;
+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 DataBlockHeader dataBlockHeader;
+private static int dataBlockHeaderLength = 
DataBlockHeaderImpl.getDataStartOffset();
+private InputStream dataBlockHeaderBuf;
+
+// 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 */;
+// the start offset in data block for this time read from ReadHandle.
+private int entryStartBlockOffset;
+// the end offset in data block for this time read from ReadHandle.
+private int entryEndBlockOffset;
+private byte[] entriesHead = new byte[entriesNumberEachRead * 
entryHeaderSize];
+private ByteBuf entriesHeadBuf = null;
+// Keep a list of all the entries ByteBuf to avoid a memory copy
+private List entriesByteBuf = null;
+private ByteBuf entryByteBuf = null;
+
+public BlockAwareSegmentInputStream(ReadHandle ledger, long startEntryId, 
int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeader = DataBlockHeaderImpl.of(blockSize, startEntryId);
+this.payloadBytesHave = 0;
+this.blockEntryCount = 0;
+this.dataBlockFullOffset = blockSize;
+this.entryStartBlockOffset = DataBlockHeaderImpl.getDataStartOffset();
+this.entryEndBlockOffset = entryStartBlockOffset;
+this.entriesByteBuf = 
Lists.newArrayListWithExpectedSize(entriesNumberEachRead);
+}
+
+// read ledger entries.
+private int readEntries() throws IOException {
+checkState(bytesReadOffset >= dataBlockHeaderLength);
+checkState(bytesReadOffset < dataBlockFullOffset);
+checkState(ledger != null);
+
+  

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-04-30 Thread GitBox
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_r184956136
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##
 @@ -0,0 +1,293 @@
+/**
+ * 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.Unpooled;
+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.apache.pulsar.broker.s3offload.DataBlockHeader;
+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 DataBlockHeader dataBlockHeader;
+private static int dataBlockHeaderLength = 
DataBlockHeaderImpl.getDataStartOffset();
+private InputStream dataBlockHeaderBuf;
+
+// 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 */;
+// the start offset in data block for this time read from ReadHandle.
+private int entryStartBlockOffset;
+// the end offset in data block for this time read from ReadHandle.
+private int entryEndBlockOffset;
+private byte[] entriesHead = new byte[entriesNumberEachRead * 
entryHeaderSize];
+private ByteBuf entriesHeadBuf = null;
+// Keep a list of all the entries ByteBuf to avoid a memory copy
+private List entriesByteBuf = null;
+private ByteBuf entryByteBuf = null;
+
+public BlockAwareSegmentInputStream(ReadHandle ledger, long startEntryId, 
int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeader = DataBlockHeaderImpl.of(blockSize, startEntryId);
+this.payloadBytesHave = 0;
+this.blockEntryCount = 0;
+this.dataBlockFullOffset = blockSize;
+this.entryStartBlockOffset = DataBlockHeaderImpl.getDataStartOffset();
+this.entryEndBlockOffset = entryStartBlockOffset;
+this.entriesByteBuf = 
Lists.newArrayListWithExpectedSize(entriesNumberEachRead);
+}
+
+// read ledger entries.
+private int readEntries() throws IOException {
+checkState(bytesReadOffset >= dataBlockHeaderLength);
+checkState(bytesReadOffset < dataBlockFullOffset);
+checkState(ledger != null);
+
+  

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-04-30 Thread GitBox
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_r184959063
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##
 @@ -0,0 +1,293 @@
+/**
+ * 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.Unpooled;
+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.apache.pulsar.broker.s3offload.DataBlockHeader;
+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 DataBlockHeader dataBlockHeader;
+private static int dataBlockHeaderLength = 
DataBlockHeaderImpl.getDataStartOffset();
+private InputStream dataBlockHeaderBuf;
+
+// 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 */;
+// the start offset in data block for this time read from ReadHandle.
+private int entryStartBlockOffset;
+// the end offset in data block for this time read from ReadHandle.
+private int entryEndBlockOffset;
+private byte[] entriesHead = new byte[entriesNumberEachRead * 
entryHeaderSize];
+private ByteBuf entriesHeadBuf = null;
+// Keep a list of all the entries ByteBuf to avoid a memory copy
+private List entriesByteBuf = null;
+private ByteBuf entryByteBuf = null;
+
+public BlockAwareSegmentInputStream(ReadHandle ledger, long startEntryId, 
int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeader = DataBlockHeaderImpl.of(blockSize, startEntryId);
+this.payloadBytesHave = 0;
+this.blockEntryCount = 0;
+this.dataBlockFullOffset = blockSize;
+this.entryStartBlockOffset = DataBlockHeaderImpl.getDataStartOffset();
+this.entryEndBlockOffset = entryStartBlockOffset;
+this.entriesByteBuf = 
Lists.newArrayListWithExpectedSize(entriesNumberEachRead);
+}
+
+// read ledger entries.
+private int readEntries() throws IOException {
+checkState(bytesReadOffset >= dataBlockHeaderLength);
+checkState(bytesReadOffset < dataBlockFullOffset);
+checkState(ledger != null);
+
+  

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-04-30 Thread GitBox
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_r184946730
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##
 @@ -0,0 +1,293 @@
+/**
+ * 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.Unpooled;
+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.apache.pulsar.broker.s3offload.DataBlockHeader;
+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 DataBlockHeader dataBlockHeader;
+private static int dataBlockHeaderLength = 
DataBlockHeaderImpl.getDataStartOffset();
+private InputStream dataBlockHeaderBuf;
+
+// 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 */;
+// the start offset in data block for this time read from ReadHandle.
+private int entryStartBlockOffset;
+// the end offset in data block for this time read from ReadHandle.
+private int entryEndBlockOffset;
+private byte[] entriesHead = new byte[entriesNumberEachRead * 
entryHeaderSize];
+private ByteBuf entriesHeadBuf = null;
+// Keep a list of all the entries ByteBuf to avoid a memory copy
+private List entriesByteBuf = null;
+private ByteBuf entryByteBuf = null;
+
+public BlockAwareSegmentInputStream(ReadHandle ledger, long startEntryId, 
int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeader = DataBlockHeaderImpl.of(blockSize, startEntryId);
+this.payloadBytesHave = 0;
+this.blockEntryCount = 0;
+this.dataBlockFullOffset = blockSize;
+this.entryStartBlockOffset = DataBlockHeaderImpl.getDataStartOffset();
+this.entryEndBlockOffset = entryStartBlockOffset;
+this.entriesByteBuf = 
Lists.newArrayListWithExpectedSize(entriesNumberEachRead);
+}
+
+// read ledger entries.
+private int readEntries() throws IOException {
+checkState(bytesReadOffset >= dataBlockHeaderLength);
+checkState(bytesReadOffset < dataBlockFullOffset);
+checkState(ledger != null);
 
 

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-04-30 Thread GitBox
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_r184951575
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##
 @@ -0,0 +1,293 @@
+/**
+ * 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.Unpooled;
+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.apache.pulsar.broker.s3offload.DataBlockHeader;
+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 DataBlockHeader dataBlockHeader;
+private static int dataBlockHeaderLength = 
DataBlockHeaderImpl.getDataStartOffset();
+private InputStream dataBlockHeaderBuf;
+
+// 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 */;
+// the start offset in data block for this time read from ReadHandle.
+private int entryStartBlockOffset;
+// the end offset in data block for this time read from ReadHandle.
+private int entryEndBlockOffset;
+private byte[] entriesHead = new byte[entriesNumberEachRead * 
entryHeaderSize];
+private ByteBuf entriesHeadBuf = null;
+// Keep a list of all the entries ByteBuf to avoid a memory copy
+private List entriesByteBuf = null;
+private ByteBuf entryByteBuf = null;
+
+public BlockAwareSegmentInputStream(ReadHandle ledger, long startEntryId, 
int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeader = DataBlockHeaderImpl.of(blockSize, startEntryId);
 
 Review comment:
   Rather than getting the DataBlockHeader object, get the input stream, as 
that's all you need to retain.


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:
us...@infra.apache.org


With regards,
Apache Git Services


[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-04-30 Thread GitBox
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_r184957838
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##
 @@ -0,0 +1,293 @@
+/**
+ * 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.Unpooled;
+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.apache.pulsar.broker.s3offload.DataBlockHeader;
+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 DataBlockHeader dataBlockHeader;
+private static int dataBlockHeaderLength = 
DataBlockHeaderImpl.getDataStartOffset();
+private InputStream dataBlockHeaderBuf;
+
+// 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 */;
+// the start offset in data block for this time read from ReadHandle.
+private int entryStartBlockOffset;
+// the end offset in data block for this time read from ReadHandle.
+private int entryEndBlockOffset;
+private byte[] entriesHead = new byte[entriesNumberEachRead * 
entryHeaderSize];
+private ByteBuf entriesHeadBuf = null;
+// Keep a list of all the entries ByteBuf to avoid a memory copy
+private List entriesByteBuf = null;
+private ByteBuf entryByteBuf = null;
+
+public BlockAwareSegmentInputStream(ReadHandle ledger, long startEntryId, 
int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeader = DataBlockHeaderImpl.of(blockSize, startEntryId);
+this.payloadBytesHave = 0;
+this.blockEntryCount = 0;
+this.dataBlockFullOffset = blockSize;
+this.entryStartBlockOffset = DataBlockHeaderImpl.getDataStartOffset();
+this.entryEndBlockOffset = entryStartBlockOffset;
+this.entriesByteBuf = 
Lists.newArrayListWithExpectedSize(entriesNumberEachRead);
+}
+
+// read ledger entries.
+private int readEntries() throws IOException {
+checkState(bytesReadOffset >= dataBlockHeaderLength);
+checkState(bytesReadOffset < dataBlockFullOffset);
+checkState(ledger != null);
+
+  

[GitHub] ivankelly commented on a change in pull request #1678: PIP-17: provide BlockAwareSegmentInputStream implementation and test

2018-04-30 Thread GitBox
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_r184958972
 
 

 ##
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/BlockAwareSegmentInputStream.java
 ##
 @@ -0,0 +1,293 @@
+/**
+ * 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.Unpooled;
+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.apache.pulsar.broker.s3offload.DataBlockHeader;
+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 DataBlockHeader dataBlockHeader;
+private static int dataBlockHeaderLength = 
DataBlockHeaderImpl.getDataStartOffset();
+private InputStream dataBlockHeaderBuf;
+
+// 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 */;
+// the start offset in data block for this time read from ReadHandle.
+private int entryStartBlockOffset;
+// the end offset in data block for this time read from ReadHandle.
+private int entryEndBlockOffset;
+private byte[] entriesHead = new byte[entriesNumberEachRead * 
entryHeaderSize];
+private ByteBuf entriesHeadBuf = null;
+// Keep a list of all the entries ByteBuf to avoid a memory copy
+private List entriesByteBuf = null;
+private ByteBuf entryByteBuf = null;
+
+public BlockAwareSegmentInputStream(ReadHandle ledger, long startEntryId, 
int blockSize) {
+this.ledger = ledger;
+this.startEntryId = startEntryId;
+this.blockSize = blockSize;
+this.dataBlockHeader = DataBlockHeaderImpl.of(blockSize, startEntryId);
+this.payloadBytesHave = 0;
+this.blockEntryCount = 0;
+this.dataBlockFullOffset = blockSize;
+this.entryStartBlockOffset = DataBlockHeaderImpl.getDataStartOffset();
+this.entryEndBlockOffset = entryStartBlockOffset;
+this.entriesByteBuf = 
Lists.newArrayListWithExpectedSize(entriesNumberEachRead);
+}
+
+// read ledger entries.
+private int readEntries() throws IOException {
+checkState(bytesReadOffset >= dataBlockHeaderLength);
+checkState(bytesReadOffset < dataBlockFullOffset);
+checkState(ledger != null);
+
+