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