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.ipc; 019 020import static org.junit.jupiter.api.Assertions.assertEquals; 021 022import java.io.IOException; 023import java.nio.ByteBuffer; 024import org.apache.commons.lang3.time.StopWatch; 025import org.apache.hadoop.hbase.Cell; 026import org.apache.hadoop.hbase.ExtendedCell; 027import org.apache.hadoop.hbase.ExtendedCellScanner; 028import org.apache.hadoop.hbase.HBaseConfiguration; 029import org.apache.hadoop.hbase.KeyValue; 030import org.apache.hadoop.hbase.PrivateCellUtil; 031import org.apache.hadoop.hbase.codec.Codec; 032import org.apache.hadoop.hbase.codec.KeyValueCodec; 033import org.apache.hadoop.hbase.io.SizedExtendedCellScanner; 034import org.apache.hadoop.hbase.nio.SingleByteBuff; 035import org.apache.hadoop.hbase.testclassification.ClientTests; 036import org.apache.hadoop.hbase.testclassification.SmallTests; 037import org.apache.hadoop.hbase.util.Bytes; 038import org.apache.hadoop.hbase.util.ClassSize; 039import org.apache.hadoop.io.compress.CompressionCodec; 040import org.apache.hadoop.io.compress.DefaultCodec; 041import org.apache.hadoop.io.compress.GzipCodec; 042import org.junit.jupiter.api.BeforeEach; 043import org.junit.jupiter.api.Tag; 044import org.junit.jupiter.api.Test; 045import org.slf4j.Logger; 046import org.slf4j.LoggerFactory; 047 048@Tag(ClientTests.TAG) 049@Tag(SmallTests.TAG) 050public class TestCellBlockBuilder { 051 052 private static final Logger LOG = LoggerFactory.getLogger(TestCellBlockBuilder.class); 053 054 private CellBlockBuilder builder; 055 056 @BeforeEach 057 public void before() { 058 this.builder = new CellBlockBuilder(HBaseConfiguration.create()); 059 } 060 061 @Test 062 public void testBuildCellBlock() throws IOException { 063 doBuildCellBlockUndoCellBlock(this.builder, new KeyValueCodec(), null); 064 doBuildCellBlockUndoCellBlock(this.builder, new KeyValueCodec(), new DefaultCodec()); 065 doBuildCellBlockUndoCellBlock(this.builder, new KeyValueCodec(), new GzipCodec()); 066 } 067 068 static void doBuildCellBlockUndoCellBlock(final CellBlockBuilder builder, final Codec codec, 069 final CompressionCodec compressor) throws IOException { 070 doBuildCellBlockUndoCellBlock(builder, codec, compressor, 10, 1, false); 071 } 072 073 static void doBuildCellBlockUndoCellBlock(final CellBlockBuilder builder, final Codec codec, 074 final CompressionCodec compressor, final int count, final int size, final boolean sized) 075 throws IOException { 076 ExtendedCell[] cells = getCells(count, size); 077 ExtendedCellScanner cellScanner = 078 sized ? getSizedCellScanner(cells) : PrivateCellUtil.createExtendedCellScanner(cells); 079 ByteBuffer bb = builder.buildCellBlock(codec, compressor, cellScanner); 080 cellScanner = 081 builder.createCellScannerReusingBuffers(codec, compressor, new SingleByteBuff(bb)); 082 int i = 0; 083 while (cellScanner.advance()) { 084 i++; 085 } 086 assertEquals(count, i); 087 } 088 089 static ExtendedCellScanner getSizedCellScanner(final ExtendedCell[] cells) { 090 int size = -1; 091 for (Cell cell : cells) { 092 size += PrivateCellUtil.estimatedSerializedSizeOf(cell); 093 } 094 final int totalSize = ClassSize.align(size); 095 final ExtendedCellScanner cellScanner = PrivateCellUtil.createExtendedCellScanner(cells); 096 return new SizedExtendedCellScanner() { 097 @Override 098 public long heapSize() { 099 return totalSize; 100 } 101 102 @Override 103 public ExtendedCell current() { 104 return cellScanner.current(); 105 } 106 107 @Override 108 public boolean advance() throws IOException { 109 return cellScanner.advance(); 110 } 111 }; 112 } 113 114 static ExtendedCell[] getCells(final int howMany) { 115 return getCells(howMany, 1024); 116 } 117 118 static ExtendedCell[] getCells(final int howMany, final int valueSize) { 119 ExtendedCell[] cells = new ExtendedCell[howMany]; 120 byte[] value = new byte[valueSize]; 121 for (int i = 0; i < howMany; i++) { 122 byte[] index = Bytes.toBytes(i); 123 KeyValue kv = new KeyValue(index, Bytes.toBytes("f"), index, value); 124 cells[i] = kv; 125 } 126 return cells; 127 } 128 129 private static final String COUNT = "--count="; 130 private static final String SIZE = "--size="; 131 132 /** 133 * Prints usage and then exits w/ passed <code>errCode</code> 134 * @param errorCode the error code to use to exit the application 135 */ 136 private static void usage(final int errorCode) { 137 System.out.println("Usage: IPCUtil [options]"); 138 System.out.println("Micro-benchmarking how changed sizes and counts work with buffer resizing"); 139 System.out.println(" --count Count of Cells"); 140 System.out.println(" --size Size of Cell values"); 141 System.out.println("Example: IPCUtil --count=1024 --size=1024"); 142 System.exit(errorCode); 143 } 144 145 private static void timerTests(final CellBlockBuilder builder, final int count, final int size, 146 final Codec codec, final CompressionCodec compressor) throws IOException { 147 final int cycles = 1000; 148 StopWatch timer = new StopWatch(); 149 timer.start(); 150 for (int i = 0; i < cycles; i++) { 151 timerTest(builder, count, size, codec, compressor, false); 152 } 153 timer.stop(); 154 LOG.info("Codec=" + codec + ", compression=" + compressor + ", sized=" + false + ", count=" 155 + count + ", size=" + size + ", + took=" + timer.getTime() + "ms"); 156 timer.reset(); 157 timer.start(); 158 for (int i = 0; i < cycles; i++) { 159 timerTest(builder, count, size, codec, compressor, true); 160 } 161 timer.stop(); 162 LOG.info("Codec=" + codec + ", compression=" + compressor + ", sized=" + true + ", count=" 163 + count + ", size=" + size + ", + took=" + timer.getTime() + "ms"); 164 } 165 166 private static void timerTest(final CellBlockBuilder builder, final int count, final int size, 167 final Codec codec, final CompressionCodec compressor, final boolean sized) throws IOException { 168 doBuildCellBlockUndoCellBlock(builder, codec, compressor, count, size, sized); 169 } 170 171 /** 172 * For running a few tests of methods herein. 173 * @param args the arguments to use for the timer test 174 * @throws IOException if creating the build fails 175 */ 176 public static void main(String[] args) throws IOException { 177 int count = 1024; 178 int size = 10240; 179 for (String arg : args) { 180 if (arg.startsWith(COUNT)) { 181 count = Integer.parseInt(arg.replace(COUNT, "")); 182 } else if (arg.startsWith(SIZE)) { 183 size = Integer.parseInt(arg.replace(SIZE, "")); 184 } else { 185 usage(1); 186 } 187 } 188 CellBlockBuilder builder = new CellBlockBuilder(HBaseConfiguration.create()); 189 timerTests(builder, count, size, new KeyValueCodec(), null); 190 timerTests(builder, count, size, new KeyValueCodec(), new DefaultCodec()); 191 timerTests(builder, count, size, new KeyValueCodec(), new GzipCodec()); 192 } 193}