001/*
002 * Licensed to the Apache Software Foundation (ASF) under one
003 * or more contributor license agreements.  See the NOTICE file
004 * distributed with this work for additional information
005 * regarding copyright ownership.  The ASF licenses this file
006 * to you under the Apache License, Version 2.0 (the
007 * "License"); you may not use this file except in compliance
008 * with the License.  You may obtain a copy of the License at
009 *
010 *     http://www.apache.org/licenses/LICENSE-2.0
011 *
012 * Unless required by applicable law or agreed to in writing, software
013 * distributed under the License is distributed on an "AS IS" BASIS,
014 * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
015 * See the License for the specific language governing permissions and
016 * limitations under the License.
017 */
018package org.apache.hadoop.hbase.io.hfile.bucket;
019
020import java.io.IOException;
021import java.util.HashMap;
022import java.util.Map;
023import java.util.concurrent.ConcurrentHashMap;
024import java.util.function.Function;
025import org.apache.hadoop.hbase.io.ByteBuffAllocator;
026import org.apache.hadoop.hbase.io.ByteBuffAllocator.Recycler;
027import org.apache.hadoop.hbase.io.hfile.BlockCacheKey;
028import org.apache.hadoop.hbase.io.hfile.BlockPriority;
029import org.apache.hadoop.hbase.io.hfile.BlockType;
030import org.apache.hadoop.hbase.io.hfile.CacheableDeserializerIdManager;
031import org.apache.hadoop.hbase.io.hfile.HFileBlock;
032import org.apache.hadoop.hbase.util.Pair;
033import org.apache.yetus.audience.InterfaceAudience;
034
035import org.apache.hbase.thirdparty.com.google.protobuf.ByteString;
036
037import org.apache.hadoop.hbase.shaded.protobuf.generated.BucketCacheProtos;
038
039@InterfaceAudience.Private
040final class BucketProtoUtils {
041  private BucketProtoUtils() {
042
043  }
044
045  static BucketCacheProtos.BucketCacheEntry toPB(BucketCache cache) {
046    return BucketCacheProtos.BucketCacheEntry.newBuilder().setCacheCapacity(cache.getMaxSize())
047      .setIoClass(cache.ioEngine.getClass().getName())
048      .setMapClass(cache.backingMap.getClass().getName())
049      .putAllDeserializers(CacheableDeserializerIdManager.save())
050      .putAllCachedFiles(toCachedPB(cache.fullyCachedFiles))
051      .setBackingMap(BucketProtoUtils.toPB(cache.backingMap))
052      .setChecksum(ByteString
053        .copyFrom(((PersistentIOEngine) cache.ioEngine).calculateChecksum(cache.getAlgorithm())))
054      .build();
055  }
056
057  private static BucketCacheProtos.BackingMap toPB(Map<BlockCacheKey, BucketEntry> backingMap) {
058    BucketCacheProtos.BackingMap.Builder builder = BucketCacheProtos.BackingMap.newBuilder();
059    for (Map.Entry<BlockCacheKey, BucketEntry> entry : backingMap.entrySet()) {
060      builder.addEntry(BucketCacheProtos.BackingMapEntry.newBuilder().setKey(toPB(entry.getKey()))
061        .setValue(toPB(entry.getValue())).build());
062    }
063    return builder.build();
064  }
065
066  private static BucketCacheProtos.BlockCacheKey toPB(BlockCacheKey key) {
067    return BucketCacheProtos.BlockCacheKey.newBuilder().setHfilename(key.getHfileName())
068      .setOffset(key.getOffset()).setPrimaryReplicaBlock(key.isPrimary())
069      .setBlockType(toPB(key.getBlockType())).build();
070  }
071
072  private static BucketCacheProtos.BlockType toPB(BlockType blockType) {
073    switch (blockType) {
074      case DATA:
075        return BucketCacheProtos.BlockType.data;
076      case META:
077        return BucketCacheProtos.BlockType.meta;
078      case TRAILER:
079        return BucketCacheProtos.BlockType.trailer;
080      case INDEX_V1:
081        return BucketCacheProtos.BlockType.index_v1;
082      case FILE_INFO:
083        return BucketCacheProtos.BlockType.file_info;
084      case LEAF_INDEX:
085        return BucketCacheProtos.BlockType.leaf_index;
086      case ROOT_INDEX:
087        return BucketCacheProtos.BlockType.root_index;
088      case BLOOM_CHUNK:
089        return BucketCacheProtos.BlockType.bloom_chunk;
090      case ENCODED_DATA:
091        return BucketCacheProtos.BlockType.encoded_data;
092      case GENERAL_BLOOM_META:
093        return BucketCacheProtos.BlockType.general_bloom_meta;
094      case INTERMEDIATE_INDEX:
095        return BucketCacheProtos.BlockType.intermediate_index;
096      case DELETE_FAMILY_BLOOM_META:
097        return BucketCacheProtos.BlockType.delete_family_bloom_meta;
098      default:
099        throw new Error("Unrecognized BlockType.");
100    }
101  }
102
103  private static BucketCacheProtos.BucketEntry toPB(BucketEntry entry) {
104    return BucketCacheProtos.BucketEntry.newBuilder().setOffset(entry.offset())
105      .setCachedTime(entry.getCachedTime()).setLength(entry.getLength())
106      .setDiskSizeWithHeader(entry.getOnDiskSizeWithHeader())
107      .setDeserialiserIndex(entry.deserializerIndex).setAccessCounter(entry.getAccessCounter())
108      .setPriority(toPB(entry.getPriority())).build();
109  }
110
111  private static BucketCacheProtos.BlockPriority toPB(BlockPriority p) {
112    switch (p) {
113      case MULTI:
114        return BucketCacheProtos.BlockPriority.multi;
115      case MEMORY:
116        return BucketCacheProtos.BlockPriority.memory;
117      case SINGLE:
118        return BucketCacheProtos.BlockPriority.single;
119      default:
120        throw new Error("Unrecognized BlockPriority.");
121    }
122  }
123
124  static ConcurrentHashMap<BlockCacheKey, BucketEntry> fromPB(Map<Integer, String> deserializers,
125    BucketCacheProtos.BackingMap backingMap, Function<BucketEntry, Recycler> createRecycler)
126    throws IOException {
127    ConcurrentHashMap<BlockCacheKey, BucketEntry> result = new ConcurrentHashMap<>();
128    for (BucketCacheProtos.BackingMapEntry entry : backingMap.getEntryList()) {
129      BucketCacheProtos.BlockCacheKey protoKey = entry.getKey();
130      BlockCacheKey key = new BlockCacheKey(protoKey.getHfilename(), protoKey.getOffset(),
131        protoKey.getPrimaryReplicaBlock(), fromPb(protoKey.getBlockType()));
132      BucketCacheProtos.BucketEntry protoValue = entry.getValue();
133      // TODO:We use ByteBuffAllocator.HEAP here, because we could not get the ByteBuffAllocator
134      // which created by RpcServer elegantly.
135      BucketEntry value = new BucketEntry(protoValue.getOffset(), protoValue.getLength(),
136        protoValue.getDiskSizeWithHeader(), protoValue.getAccessCounter(),
137        protoValue.getCachedTime(),
138        protoValue.getPriority() == BucketCacheProtos.BlockPriority.memory, createRecycler,
139        ByteBuffAllocator.HEAP);
140      // This is the deserializer that we stored
141      int oldIndex = protoValue.getDeserialiserIndex();
142      String deserializerClass = deserializers.get(oldIndex);
143      if (deserializerClass == null) {
144        throw new IOException("Found deserializer index without matching entry.");
145      }
146      // Convert it to the identifier for the deserializer that we have in this runtime
147      if (deserializerClass.equals(HFileBlock.BlockDeserializer.class.getName())) {
148        int actualIndex = HFileBlock.BLOCK_DESERIALIZER.getDeserializerIdentifier();
149        value.deserializerIndex = (byte) actualIndex;
150      } else {
151        // We could make this more plugable, but right now HFileBlock is the only implementation
152        // of Cacheable outside of tests, so this might not ever matter.
153        throw new IOException("Unknown deserializer class found: " + deserializerClass);
154      }
155      result.put(key, value);
156    }
157    return result;
158  }
159
160  private static BlockType fromPb(BucketCacheProtos.BlockType blockType) {
161    switch (blockType) {
162      case data:
163        return BlockType.DATA;
164      case meta:
165        return BlockType.META;
166      case trailer:
167        return BlockType.TRAILER;
168      case index_v1:
169        return BlockType.INDEX_V1;
170      case file_info:
171        return BlockType.FILE_INFO;
172      case leaf_index:
173        return BlockType.LEAF_INDEX;
174      case root_index:
175        return BlockType.ROOT_INDEX;
176      case bloom_chunk:
177        return BlockType.BLOOM_CHUNK;
178      case encoded_data:
179        return BlockType.ENCODED_DATA;
180      case general_bloom_meta:
181        return BlockType.GENERAL_BLOOM_META;
182      case intermediate_index:
183        return BlockType.INTERMEDIATE_INDEX;
184      case delete_family_bloom_meta:
185        return BlockType.DELETE_FAMILY_BLOOM_META;
186      default:
187        throw new Error("Unrecognized BlockType.");
188    }
189  }
190
191  static Map<String, BucketCacheProtos.RegionFileSizeMap>
192    toCachedPB(Map<String, Pair<String, Long>> prefetchedHfileNames) {
193    Map<String, BucketCacheProtos.RegionFileSizeMap> tmpMap = new HashMap<>();
194    prefetchedHfileNames.forEach((hfileName, regionPrefetchMap) -> {
195      BucketCacheProtos.RegionFileSizeMap tmpRegionFileSize =
196        BucketCacheProtos.RegionFileSizeMap.newBuilder().setRegionName(regionPrefetchMap.getFirst())
197          .setRegionCachedSize(regionPrefetchMap.getSecond()).build();
198      tmpMap.put(hfileName, tmpRegionFileSize);
199    });
200    return tmpMap;
201  }
202
203  static Map<String, Pair<String, Long>>
204    fromPB(Map<String, BucketCacheProtos.RegionFileSizeMap> prefetchHFileNames) {
205    Map<String, Pair<String, Long>> hfileMap = new HashMap<>();
206    prefetchHFileNames.forEach((hfileName, regionPrefetchMap) -> {
207      hfileMap.put(hfileName,
208        new Pair<>(regionPrefetchMap.getRegionName(), regionPrefetchMap.getRegionCachedSize()));
209    });
210    return hfileMap;
211  }
212}