ivankelly commented on a change in pull request #1593: PIP-17:  the part of 
index block for offload.
URL: https://github.com/apache/incubator-pulsar/pull/1593#discussion_r185009429
 
 

 ##########
 File path: 
pulsar-broker/src/main/java/org/apache/pulsar/broker/s3offload/impl/OffloadIndexBlockImpl.java
 ##########
 @@ -0,0 +1,336 @@
+/**
+ * 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.Maps;
+import io.netty.buffer.ByteBuf;
+import io.netty.buffer.ByteBufInputStream;
+import io.netty.buffer.PooledByteBufAllocator;
+import io.netty.util.Recycler;
+import io.netty.util.Recycler.Handle;
+import java.io.DataInputStream;
+import java.io.IOException;
+import java.io.InputStream;
+import java.util.ArrayList;
+import java.util.List;
+import java.util.Map;
+import java.util.NavigableMap;
+import java.util.TreeMap;
+import java.util.stream.Collectors;
+import org.apache.bookkeeper.client.BookKeeper;
+import org.apache.bookkeeper.client.api.DigestType;
+import org.apache.bookkeeper.client.api.LedgerMetadata;
+import org.apache.bookkeeper.net.BookieSocketAddress;
+import org.apache.bookkeeper.proto.DataFormats;
+import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat;
+import org.apache.bookkeeper.proto.DataFormats.LedgerMetadataFormat.State;
+import org.apache.bookkeeper.shaded.com.google.protobuf.ByteString;
+import org.apache.pulsar.broker.s3offload.OffloadIndexBlock;
+import org.apache.pulsar.broker.s3offload.OffloadIndexEntry;
+import org.slf4j.Logger;
+import org.slf4j.LoggerFactory;
+
+public class OffloadIndexBlockImpl implements OffloadIndexBlock {
+    private static final Logger log = 
LoggerFactory.getLogger(OffloadIndexBlockImpl.class);
+
+    private static final int indexMagicWord = 0xDE47DE47;
+
+    private LedgerMetadata segmentMetadata;
+    private TreeMap<Long, OffloadIndexEntryImpl> indexEntries;
+
+    private final Handle<OffloadIndexBlockImpl> recyclerHandle;
+
+    private static final Recycler<OffloadIndexBlockImpl> RECYCLER = new 
Recycler<OffloadIndexBlockImpl>() {
+        @Override
+        protected OffloadIndexBlockImpl 
newObject(Recycler.Handle<OffloadIndexBlockImpl> handle) {
+            return new OffloadIndexBlockImpl(handle);
+        }
+    };
+
+    private OffloadIndexBlockImpl(Handle<OffloadIndexBlockImpl> 
recyclerHandle) {
+        this.recyclerHandle = recyclerHandle;
+    }
+
+    public static OffloadIndexBlockImpl get(LedgerMetadata metadata, 
List<OffloadIndexEntryImpl> entries) {
+        OffloadIndexBlockImpl block = RECYCLER.get();
+        block.indexEntries = Maps.newTreeMap();
+        entries.forEach(entry -> 
block.indexEntries.putIfAbsent(entry.getEntryId(), entry));
+        checkState(entries.size() == block.indexEntries.size());
+        block.segmentMetadata = metadata;
+        return block;
+    }
+
+    public static OffloadIndexBlockImpl get(InputStream stream) throws 
IOException {
+        OffloadIndexBlockImpl block = RECYCLER.get();
+        block.indexEntries = Maps.newTreeMap();
+        block.fromStream(stream);
+        return block;
+    }
+
+    public void recycle() {
+        segmentMetadata = null;
+        indexEntries.clear();
+        indexEntries = null;
+        if (recyclerHandle != null) {
+            recyclerHandle.recycle(this);
+        }
+    }
+
+    @Override
+    public OffloadIndexEntry getIndexEntryForEntry(long messageEntryId) {
+        if(messageEntryId > segmentMetadata.getLastEntryId()) {
+            log.warn("Try to get entry: {}, which beyond lastEntryId {}, 
return null",
+                messageEntryId, segmentMetadata.getLastEntryId());
+            return null;
+        }
+        // find the greatest mapping Id whose entryId <= messageEntryId
+        return this.indexEntries.floorEntry(messageEntryId).getValue();
+    }
+
+    @Override
+    public int getEntryCount() {
+        return this.indexEntries.size();
+    }
+
+    @Override
+    public LedgerMetadata getLedgerMetadata() {
+        return this.segmentMetadata;
+    }
+
+    private static byte[] buildLedgerMetadataFormat(LedgerMetadata metadata) {
+        LedgerMetadataFormat.Builder builder = 
LedgerMetadataFormat.newBuilder();
+        builder.setQuorumSize(metadata.getWriteQuorumSize())
+            .setAckQuorumSize(metadata.getAckQuorumSize())
+            .setEnsembleSize(metadata.getEnsembleSize())
+            .setLength(metadata.getLength())
+            .setState(metadata.isClosed() ? LedgerMetadataFormat.State.CLOSED 
: LedgerMetadataFormat.State.OPEN)
+            .setLastEntryId(metadata.getLastEntryId())
+            .setCtime(metadata.getCtime())
+            .setDigestType(BookKeeper.DigestType.toProtoDigestType(
+                
BookKeeper.DigestType.fromApiDigestType(metadata.getDigestType())));
+
+        for (Map.Entry<String, byte[]> e : 
metadata.getCustomMetadata().entrySet()) {
+            builder.addCustomMetadataBuilder()
+                
.setKey(e.getKey()).setValue(ByteString.copyFrom(e.getValue()));
+        }
+
+        for (Map.Entry<Long, ? extends List<BookieSocketAddress>> e : 
metadata.getAllEnsembles().entrySet()) {
+            builder.addSegmentBuilder()
+                .setFirstEntryId(e.getKey())
+                .addAllEnsembleMember(e.getValue().stream().map(a -> 
a.toString()).collect(Collectors.toList()));
+        }
+
+        return builder.build().toByteArray();
+    }
+
+    /**
+     * Get the content of the index block as InputStream.
+     * Read out in format:
+     *   | index_magic_header | index_block_len | index_entry_count |
+     *   |segment_metadata_len | segment metadata | index entries |
+     */
+    @Override
+    public InputStream toStream() throws IOException {
+        int indexBlockLength;
+        int segmentMetadataLength;
+        int indexEntryCount = this.indexEntries.size();
+
+        byte[] ledgerMetadataByte = 
buildLedgerMetadataFormat(this.segmentMetadata);
+        segmentMetadataLength = ledgerMetadataByte.length;
+
+        indexBlockLength = 4 /* magic header */
+            + 4 /* index block length */
+            + 4 /* segment metadata length */
+            + 4 /* index entry count */
+            + segmentMetadataLength
+            + indexEntryCount * (8 + 4 + 8); /* messageEntryId + blockPartId + 
blockOffset */
+
+        ByteBuf out = PooledByteBufAllocator.DEFAULT.buffer(indexBlockLength, 
indexBlockLength);
+
+        out.writeInt(indexMagicWord)
+            .writeInt(indexBlockLength)
+            .writeInt(segmentMetadataLength)
+            .writeInt(indexEntryCount);
+
+        // write metadata
+        checkState(out.writerIndex() == 16);
+        out.writeBytes(ledgerMetadataByte);
+
+        // write entries
+        this.indexEntries.entrySet().forEach(entry ->
+            out.writeLong(entry.getValue().getEntryId())
+                .writeInt(entry.getValue().getPartId())
+                .writeLong(entry.getValue().getOffset()));
+
+        return new ByteBufInputStream(out, true);
+    }
+
+    static private class InternalLedgerMetadata implements LedgerMetadata {
+        private LedgerMetadataFormat ledgerMetadataFormat;
+
+        private int ensembleSize;
+        private int writeQuorumSize;
+        private int ackQuorumSize;
+        private long lastEntryId;
+        private long length;
+        private DataFormats.LedgerMetadataFormat.DigestType digestType;
+        private long ctime;
+        private State state;
+        private Map<String, byte[]> customMetadata = Maps.newHashMap();
+        private TreeMap<Long, ArrayList<BookieSocketAddress>> ensembles = new 
TreeMap<Long, ArrayList<BookieSocketAddress>>();
+
+        InternalLedgerMetadata(LedgerMetadataFormat ledgerMetadataFormat) {
+            this.ensembleSize = ledgerMetadataFormat.getEnsembleSize();
+            this.writeQuorumSize = ledgerMetadataFormat.getQuorumSize();
+            this.ackQuorumSize = ledgerMetadataFormat.getAckQuorumSize();
+            this.lastEntryId = ledgerMetadataFormat.getLastEntryId();
+            this.length = ledgerMetadataFormat.getLength();
+            this.digestType = ledgerMetadataFormat.getDigestType();
+            this.ctime = ledgerMetadataFormat.getCtime();
+            this.state = ledgerMetadataFormat.getState();
+
+            if (ledgerMetadataFormat.getCustomMetadataCount() > 0) {
+                ledgerMetadataFormat.getCustomMetadataList().forEach(
+                    entry -> this.customMetadata.put(entry.getKey(), 
entry.getValue().toByteArray()));
+            }
+
+            ledgerMetadataFormat.getSegmentList().forEach(segment -> {
+                ArrayList<BookieSocketAddress> addressArrayList = new 
ArrayList<BookieSocketAddress>();
+                segment.getEnsembleMemberList().forEach(address -> {
+                    try {
+                        addressArrayList.add(new BookieSocketAddress(address));
+                    } catch (IOException e) {
+                        log.error("Exception when create BookieSocketAddress. 
", e);
+                    }
+                });
+                this.ensembles.put(segment.getFirstEntryId(), 
addressArrayList);
+            });
+        }
+
+        @Override
+        public int getEnsembleSize() {
+            return this.ensembleSize;
+        }
+
+        @Override
+        public int getWriteQuorumSize() {
+            return this.writeQuorumSize;
+        }
+
+        @Override
+        public int getAckQuorumSize() {
+            return this.ackQuorumSize;
+        }
+
+        @Override
+        public long getLastEntryId() {
+            return this.lastEntryId;
+        }
+
+        @Override
+        public long getLength() {
+            return this.length;
+        }
+
+        @Override
+        public DigestType getDigestType() {
+            switch (this.digestType) {
+                case HMAC:
+                    return DigestType.MAC;
+                case CRC32:
+                    return DigestType.CRC32;
+                case CRC32C:
+                    return DigestType.CRC32C;
+                case DUMMY:
+                    return DigestType.DUMMY;
+                default:
+                    throw new IllegalArgumentException("Unable to convert 
digest type " + digestType);
+            }
+        }
+
+        @Override
+        public long getCtime() {
+            return this.ctime;
+        }
+
+        @Override
+        public boolean isClosed() {
+            return this.state == State.CLOSED;
+        }
+
+        @Override
+        public Map<String, byte[]> getCustomMetadata() {
+            return this.customMetadata;
+        }
+
+        @Override
+        public List<BookieSocketAddress> getEnsembleAt(long entryId) {
+            return ensembles.get(ensembles.headMap(entryId + 1).lastKey());
+        }
+
+        @Override
+        public NavigableMap<Long, ? extends List<BookieSocketAddress>> 
getAllEnsembles() {
+            return this.ensembles;
+        }
+    }
+
+    private static LedgerMetadata parseLedgerMetadata(byte[] bytes) throws 
IOException {
+        LedgerMetadataFormat.Builder builder = 
LedgerMetadataFormat.newBuilder();
+        builder.mergeFrom(bytes);
+        return new InternalLedgerMetadata(builder.build());
+    }
+
+    private OffloadIndexBlock fromStream(InputStream stream) throws 
IOException {
+        DataInputStream dis = new DataInputStream(stream);
+        int magic = dis.readInt();
+        checkState(magic == this.indexMagicWord);
 
 Review comment:
   Throw a checked exception

----------------------------------------------------------------
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

Reply via email to