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.regionserver.compactions; 019 020import java.util.ArrayList; 021import java.util.List; 022import java.util.Optional; 023import org.apache.hadoop.conf.Configuration; 024import org.apache.hadoop.hbase.ArrayBackedTag; 025import org.apache.hadoop.hbase.Cell; 026import org.apache.hadoop.hbase.CellUtil; 027import org.apache.hadoop.hbase.ExtendedCell; 028import org.apache.hadoop.hbase.PrivateCellUtil; 029import org.apache.hadoop.hbase.Tag; 030import org.apache.hadoop.hbase.TagType; 031import org.apache.hadoop.hbase.util.Bytes; 032import org.apache.yetus.audience.InterfaceAudience; 033 034/** 035 * An extension of DateTieredCompactor, overriding the decorateCells method to allow for custom 036 * values to be used for the different file tiers during compaction. 037 */ 038@InterfaceAudience.Private 039public class CustomCellTieringValueProvider implements CustomTieredCompactor.TieringValueProvider { 040 public static final String TIERING_CELL_QUALIFIER = "TIERING_CELL_QUALIFIER"; 041 private byte[] tieringQualifier; 042 043 @Override 044 public void init(Configuration conf) throws Exception { 045 tieringQualifier = Bytes.toBytes(conf.get(TIERING_CELL_QUALIFIER)); 046 } 047 048 @Override 049 public List<ExtendedCell> decorateCells(List<ExtendedCell> cells) { 050 // if no tiering qualifier properly set, skips the whole flow 051 if (tieringQualifier != null) { 052 byte[] tieringValue = null; 053 // first iterates through the cells within a row, to find the tiering value for the row 054 for (Cell cell : cells) { 055 if (CellUtil.matchingQualifier(cell, tieringQualifier)) { 056 tieringValue = CellUtil.cloneValue(cell); 057 break; 058 } 059 } 060 if (tieringValue == null) { 061 tieringValue = Bytes.toBytes(Long.MAX_VALUE); 062 } 063 // now apply the tiering value as a tag to all cells within the row 064 Tag tieringValueTag = new ArrayBackedTag(TagType.CELL_VALUE_TIERING_TAG_TYPE, tieringValue); 065 List<ExtendedCell> newCells = new ArrayList<>(cells.size()); 066 for (ExtendedCell cell : cells) { 067 List<Tag> tags = PrivateCellUtil.getTags(cell); 068 tags.add(tieringValueTag); 069 newCells.add(PrivateCellUtil.createCell(cell, tags)); 070 } 071 return newCells; 072 } else { 073 return cells; 074 } 075 } 076 077 @Override 078 public long getTieringValue(ExtendedCell cell) { 079 Optional<Tag> tagOptional = PrivateCellUtil.getTag(cell, TagType.CELL_VALUE_TIERING_TAG_TYPE); 080 if (tagOptional.isPresent()) { 081 Tag tag = tagOptional.get(); 082 return Bytes.toLong(tag.getValueByteBuffer().array(), tag.getValueOffset(), 083 tag.getValueLength()); 084 } 085 return Long.MAX_VALUE; 086 } 087}