View Javadoc

1   /*
2    * Licensed to the Apache Software Foundation (ASF) under one
3    * or more contributor license agreements.  See the NOTICE file
4    * distributed with this work for additional information
5    * regarding copyright ownership.  The ASF licenses this file
6    * to you under the Apache License, Version 2.0 (the
7    * "License"); you may not use this file except in compliance
8    * with the License.  You may obtain a copy of the License at
9    *
10   *     http://www.apache.org/licenses/LICENSE-2.0
11   *
12   * Unless required by applicable law or agreed to in writing, software
13   * distributed under the License is distributed on an "AS IS" BASIS,
14   * WITHOUT WARRANTIES OR CONDITIONS OF ANY KIND, either express or implied.
15   * See the License for the specific language governing permissions and
16   * limitations under the License.
17   */
18  
19  package org.apache.hadoop.hbase.codec.prefixtree;
20  
21  import java.io.DataInputStream;
22  import java.io.DataOutputStream;
23  import java.io.IOException;
24  import java.nio.ByteBuffer;
25  
26  import org.apache.hadoop.hbase.classification.InterfaceAudience;
27  import org.apache.hadoop.hbase.Cell;
28  import org.apache.hadoop.hbase.KeyValue;
29  import org.apache.hadoop.hbase.KeyValue.KVComparator;
30  import org.apache.hadoop.hbase.KeyValue.MetaComparator;
31  import org.apache.hadoop.hbase.KeyValue.RawBytesComparator;
32  import org.apache.hadoop.hbase.KeyValueUtil;
33  import org.apache.hadoop.hbase.codec.prefixtree.decode.DecoderFactory;
34  import org.apache.hadoop.hbase.codec.prefixtree.decode.PrefixTreeArraySearcher;
35  import org.apache.hadoop.hbase.codec.prefixtree.encode.EncoderFactory;
36  import org.apache.hadoop.hbase.codec.prefixtree.encode.PrefixTreeEncoder;
37  import org.apache.hadoop.hbase.codec.prefixtree.scanner.CellSearcher;
38  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoder;
39  import org.apache.hadoop.hbase.io.encoding.DataBlockEncoding;
40  import org.apache.hadoop.hbase.io.encoding.EncodingState;
41  import org.apache.hadoop.hbase.io.encoding.HFileBlockDecodingContext;
42  import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultDecodingContext;
43  import org.apache.hadoop.hbase.io.encoding.HFileBlockDefaultEncodingContext;
44  import org.apache.hadoop.hbase.io.encoding.HFileBlockEncodingContext;
45  import org.apache.hadoop.hbase.io.hfile.BlockType;
46  import org.apache.hadoop.hbase.io.hfile.HFileContext;
47  import org.apache.hadoop.hbase.util.ByteBufferUtils;
48  import org.apache.hadoop.io.WritableUtils;
49  
50  /**
51   * <p>
52   * This class is created via reflection in DataBlockEncoding enum. Update the enum if class name or
53   * package changes.
54   * </p>
55   * PrefixTreeDataBlockEncoder implementation of DataBlockEncoder. This is the primary entry point
56   * for PrefixTree encoding and decoding. Encoding is delegated to instances of
57   * {@link PrefixTreeEncoder}, and decoding is delegated to instances of
58   * {@link org.apache.hadoop.hbase.codec.prefixtree.scanner.CellSearcher}. Encoder and decoder instances are
59   * created and recycled by static PtEncoderFactory and PtDecoderFactory.
60   * @deprecated Since 1.2.7. Will be removed in 2.0.0. Write the dev list if you would like to take on the
61   * maintainence of this facility.
62   */
63  @Deprecated
64  @InterfaceAudience.Private
65  public class PrefixTreeCodec implements DataBlockEncoder{
66  
67    /**
68     * no-arg constructor for reflection
69     */
70    public PrefixTreeCodec() {
71    }
72  
73    @Override
74    public ByteBuffer decodeKeyValues(DataInputStream source, HFileBlockDecodingContext decodingCtx)
75        throws IOException {
76      return decodeKeyValues(source, 0, 0, decodingCtx);
77    }
78  
79  
80    /**
81     * I don't think this method is called during normal HBase operation, so efficiency is not
82     * important.
83     */
84    public ByteBuffer decodeKeyValues(DataInputStream source, int allocateHeaderLength,
85        int skipLastBytes, HFileBlockDecodingContext decodingCtx) throws IOException {
86      ByteBuffer sourceAsBuffer = ByteBufferUtils.drainInputStreamToBuffer(source);// waste
87      sourceAsBuffer.mark();
88      PrefixTreeBlockMeta blockMeta = new PrefixTreeBlockMeta(sourceAsBuffer);
89      sourceAsBuffer.rewind();
90      int numV1BytesWithHeader = allocateHeaderLength + blockMeta.getNumKeyValueBytes();
91      byte[] keyValueBytesWithHeader = new byte[numV1BytesWithHeader];
92      ByteBuffer result = ByteBuffer.wrap(keyValueBytesWithHeader);
93      result.rewind();
94      CellSearcher searcher = null;
95      try {
96        boolean includesMvcc = decodingCtx.getHFileContext().isIncludesMvcc();
97        searcher = DecoderFactory.checkOut(sourceAsBuffer, includesMvcc);
98        while (searcher.advance()) {
99          KeyValue currentCell = KeyValueUtil.copyToNewKeyValue(searcher.current());
100         // needs to be modified for DirectByteBuffers. no existing methods to
101         // write VLongs to byte[]
102         int offset = result.arrayOffset() + result.position();
103         System.arraycopy(currentCell.getBuffer(), currentCell.getOffset(), result.array(), offset,
104             currentCell.getLength());
105         int keyValueLength = KeyValueUtil.length(currentCell);
106         ByteBufferUtils.skip(result, keyValueLength);
107         offset += keyValueLength;
108         if (includesMvcc) {
109           ByteBufferUtils.writeVLong(result, currentCell.getMvccVersion());
110         }
111       }
112       result.position(result.limit());//make it appear as if we were appending
113       return result;
114     } finally {
115       DecoderFactory.checkIn(searcher);
116     }
117   }
118 
119 
120   @Override
121   public ByteBuffer getFirstKeyInBlock(ByteBuffer block) {
122     block.rewind();
123     PrefixTreeArraySearcher searcher = null;
124     try {
125       // should i includeMemstoreTS (second argument)?  i think PrefixKeyDeltaEncoder is, so i will
126       searcher = DecoderFactory.checkOut(block, true);
127       if (!searcher.positionAtFirstCell()) {
128         return null;
129       }
130       return KeyValueUtil.copyKeyToNewByteBuffer(searcher.current());
131     } finally {
132       DecoderFactory.checkIn(searcher);
133     }
134   }
135 
136   @Override
137   public HFileBlockEncodingContext newDataBlockEncodingContext(
138       DataBlockEncoding encoding, byte[] header, HFileContext meta) {
139     if(DataBlockEncoding.PREFIX_TREE != encoding){
140       //i'm not sure why encoding is in the interface.  Each encoder implementation should probably
141       //know it's encoding type
142       throw new IllegalArgumentException("only DataBlockEncoding.PREFIX_TREE supported");
143     }
144     return new HFileBlockDefaultEncodingContext(encoding, header, meta);
145   }
146 
147   @Override
148   public HFileBlockDecodingContext newDataBlockDecodingContext(HFileContext meta) {
149     return new HFileBlockDefaultDecodingContext(meta);
150   }
151 
152   /**
153    * Is this the correct handling of an illegal comparator?  How to prevent that from getting all
154    * the way to this point.
155    */
156   @Override
157   public EncodedSeeker createSeeker(KVComparator comparator, HFileBlockDecodingContext decodingCtx) {
158     if (comparator instanceof RawBytesComparator){
159       throw new IllegalArgumentException("comparator must be KeyValue.KeyComparator");
160     } else if (comparator instanceof MetaComparator){
161       throw new IllegalArgumentException("DataBlockEncoding.PREFIX_TREE not compatible with hbase:meta "
162           +"table");
163     }
164 
165     return new PrefixTreeSeeker(decodingCtx.getHFileContext().isIncludesMvcc());
166   }
167 
168   @Override
169   public int encode(Cell cell, HFileBlockEncodingContext encodingCtx, DataOutputStream out)
170       throws IOException {
171     PrefixTreeEncodingState state = (PrefixTreeEncodingState) encodingCtx.getEncodingState();
172     PrefixTreeEncoder builder = state.builder;
173     builder.write(cell);
174     int size = KeyValueUtil.length(cell);
175     if (encodingCtx.getHFileContext().isIncludesMvcc()) {
176       size += WritableUtils.getVIntSize(cell.getSequenceId());
177     }
178     return size;
179   }
180 
181   private static class PrefixTreeEncodingState extends EncodingState {
182     PrefixTreeEncoder builder = null;
183   }
184 
185   @Override
186   public void startBlockEncoding(HFileBlockEncodingContext blkEncodingCtx, DataOutputStream out)
187       throws IOException {
188     if (blkEncodingCtx.getClass() != HFileBlockDefaultEncodingContext.class) {
189       throw new IOException(this.getClass().getName() + " only accepts "
190           + HFileBlockDefaultEncodingContext.class.getName() + " as the " + "encoding context.");
191     }
192 
193     HFileBlockDefaultEncodingContext encodingCtx = 
194         (HFileBlockDefaultEncodingContext) blkEncodingCtx;
195     encodingCtx.prepareEncoding(out);
196 
197     PrefixTreeEncoder builder = EncoderFactory.checkOut(out, encodingCtx.getHFileContext()
198         .isIncludesMvcc());
199     PrefixTreeEncodingState state = new PrefixTreeEncodingState();
200     state.builder = builder;
201     blkEncodingCtx.setEncodingState(state);
202   }
203 
204   @Override
205   public void endBlockEncoding(HFileBlockEncodingContext encodingCtx, DataOutputStream out,
206       byte[] uncompressedBytesWithHeader) throws IOException {
207     PrefixTreeEncodingState state = (PrefixTreeEncodingState) encodingCtx.getEncodingState();
208     PrefixTreeEncoder builder = state.builder;
209     builder.flush();
210     EncoderFactory.checkIn(builder);
211     // do i need to check this, or will it always be DataBlockEncoding.PREFIX_TREE?
212     if (encodingCtx.getDataBlockEncoding() != DataBlockEncoding.NONE) {
213       encodingCtx.postEncoding(BlockType.ENCODED_DATA);
214     } else {
215       encodingCtx.postEncoding(BlockType.DATA);
216     }
217   }
218 }