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   * The portion of this file denoted by 'Copied from com.google.protobuf.CodedOutputStream'
19   * is from Protocol Buffers v2.5.0 under the following license
20   *
21   * Copyright 2008 Google Inc.  All rights reserved.
22   * http://code.google.com/p/protobuf/
23   *
24   * Redistribution and use in source and binary forms, with or without
25   * modification, are permitted provided that the following conditions are
26   * met:
27   *
28   *     * Redistributions of source code must retain the above copyright
29   * notice, this list of conditions and the following disclaimer.
30   *     * Redistributions in binary form must reproduce the above
31   * copyright notice, this list of conditions and the following disclaimer
32   * in the documentation and/or other materials provided with the
33   * distribution.
34   *     * Neither the name of Google Inc. nor the names of its
35   * contributors may be used to endorse or promote products derived from
36   * this software without specific prior written permission.
37   *
38   * THIS SOFTWARE IS PROVIDED BY THE COPYRIGHT HOLDERS AND CONTRIBUTORS
39   * "AS IS" AND ANY EXPRESS OR IMPLIED WARRANTIES, INCLUDING, BUT NOT
40   * LIMITED TO, THE IMPLIED WARRANTIES OF MERCHANTABILITY AND FITNESS FOR
41   * A PARTICULAR PURPOSE ARE DISCLAIMED. IN NO EVENT SHALL THE COPYRIGHT
42   * OWNER OR CONTRIBUTORS BE LIABLE FOR ANY DIRECT, INDIRECT, INCIDENTAL,
43   * SPECIAL, EXEMPLARY, OR CONSEQUENTIAL DAMAGES (INCLUDING, BUT NOT
44   * LIMITED TO, PROCUREMENT OF SUBSTITUTE GOODS OR SERVICES; LOSS OF USE,
45   * DATA, OR PROFITS; OR BUSINESS INTERRUPTION) HOWEVER CAUSED AND ON ANY
46   * THEORY OF LIABILITY, WHETHER IN CONTRACT, STRICT LIABILITY, OR TORT
47   * (INCLUDING NEGLIGENCE OR OTHERWISE) ARISING IN ANY WAY OUT OF THE USE
48   * OF THIS SOFTWARE, EVEN IF ADVISED OF THE POSSIBILITY OF SUCH DAMAGE.
49   */
50  
51  package org.apache.hadoop.hbase.util;
52  
53  import org.apache.hadoop.hbase.classification.InterfaceAudience;
54  import org.apache.hadoop.hbase.classification.InterfaceStability;
55  
56  /**
57   * Extends the basic {@link AbstractPositionedByteRange} implementation with
58   * position support and it is a mutable version. {@code position} is considered transient,
59   * not fundamental to the definition of the range, and does not participate in
60   * {@link #compareTo(ByteRange)}, {@link #hashCode()}, or
61   * {@link #equals(Object)}. {@code Position} is retained by copy operations.
62   */
63  @InterfaceAudience.Public
64  @InterfaceStability.Evolving
65  @edu.umd.cs.findbugs.annotations.SuppressWarnings("EQ_DOESNT_OVERRIDE_EQUALS")
66  public class SimplePositionedMutableByteRange extends AbstractPositionedByteRange {
67    /**
68     * Create a new {@code PositionedByteRange} lacking a backing array and with
69     * an undefined viewport.
70     */
71    public SimplePositionedMutableByteRange() {
72      super();
73    }
74  
75    /**
76     * Create a new {@code PositionedByteRange} over a new backing array of size
77     * {@code capacity}. The range's offset and length are 0 and {@code capacity},
78     * respectively.
79     * 
80     * @param capacity
81     *          the size of the backing array.
82     */
83    public SimplePositionedMutableByteRange(int capacity) {
84      this(new byte[capacity]);
85    }
86  
87    /**
88     * Create a new {@code PositionedByteRange} over the provided {@code bytes}.
89     * 
90     * @param bytes
91     *          The array to wrap.
92     */
93    public SimplePositionedMutableByteRange(byte[] bytes) {
94      set(bytes);
95    }
96  
97    /**
98     * Create a new {@code PositionedByteRange} over the provided {@code bytes}.
99     * 
100    * @param bytes
101    *          The array to wrap.
102    * @param offset
103    *          The offset into {@code bytes} considered the beginning of this
104    *          range.
105    * @param length
106    *          The length of this range.
107    */
108   public SimplePositionedMutableByteRange(byte[] bytes, int offset, int length) {
109     set(bytes, offset, length);
110   }
111 
112   @Override
113   public PositionedByteRange unset() {
114     this.position = 0;
115     clearHashCache();
116     bytes = null;
117     offset = 0;
118     length = 0;
119     return this;
120   }
121 
122   @Override
123   public PositionedByteRange set(int capacity) {
124     this.position = 0;
125     super.set(capacity);
126     this.limit = capacity;
127     return this;
128   }
129 
130   @Override
131   public PositionedByteRange set(byte[] bytes) {
132     this.position = 0;
133     super.set(bytes);
134     this.limit = bytes.length;
135     return this;
136   }
137 
138   @Override
139   public PositionedByteRange set(byte[] bytes, int offset, int length) {
140     this.position = 0;
141     super.set(bytes, offset, length);
142     limit = length;
143     return this;
144   }
145 
146   /**
147    * Update the beginning of this range. {@code offset + length} may not be
148    * greater than {@code bytes.length}. Resets {@code position} to 0.
149    * 
150    * @param offset
151    *          the new start of this range.
152    * @return this.
153    */
154   @Override
155   public PositionedByteRange setOffset(int offset) {
156     this.position = 0;
157     super.setOffset(offset);
158     return this;
159   }
160 
161   /**
162    * Update the length of this range. {@code offset + length} should not be
163    * greater than {@code bytes.length}. If {@code position} is greater than the
164    * new {@code length}, sets {@code position} to {@code length}.
165    * 
166    * @param length
167    *          The new length of this range.
168    * @return this.
169    */
170   @Override
171   public PositionedByteRange setLength(int length) {
172     this.position = Math.min(position, length);
173     super.setLength(length);
174     return this;
175   }
176 
177   @Override
178   public PositionedByteRange put(byte val) {
179     put(position++, val);
180     return this;
181   }
182 
183   @Override
184   public PositionedByteRange put(byte[] val) {
185     if (0 == val.length)
186       return this;
187     return this.put(val, 0, val.length);
188   }
189 
190   @Override
191   public PositionedByteRange put(byte[] val, int offset, int length) {
192     if (0 == length)
193       return this;
194     put(position, val, offset, length);
195     this.position += length;
196     return this;
197   }
198 
199   @Override
200   public PositionedByteRange get(int index, byte[] dst) {
201     super.get(index, dst);
202     return this;
203   }
204 
205   @Override
206   public PositionedByteRange get(int index, byte[] dst, int offset, int length) {
207     super.get(index, dst, offset, length);
208     return this;
209   }
210 
211   @Override
212   public PositionedByteRange put(int index, byte val) {
213     bytes[offset + index] = val;
214     return this;
215   }
216 
217   @Override
218   public PositionedByteRange put(int index, byte[] val) {
219     if (0 == val.length)
220       return this;
221     return put(index, val, 0, val.length);
222   }
223 
224   @Override
225   public PositionedByteRange put(int index, byte[] val, int offset, int length) {
226     if (0 == length)
227       return this;
228     System.arraycopy(val, offset, this.bytes, this.offset + index, length);
229     return this;
230   }
231 
232   @Override
233   public PositionedByteRange deepCopy() {
234     SimplePositionedMutableByteRange clone = new SimplePositionedMutableByteRange(
235         deepCopyToNewArray());
236     clone.position = this.position;
237     return clone;
238   }
239 
240   @Override
241   public PositionedByteRange shallowCopy() {
242     SimplePositionedMutableByteRange clone = new SimplePositionedMutableByteRange(bytes, offset,
243         length);
244     clone.position = this.position;
245     return clone;
246   }
247 
248   @Override
249   public PositionedByteRange shallowCopySubRange(int innerOffset, int copyLength) {
250     SimplePositionedMutableByteRange clone = new SimplePositionedMutableByteRange(bytes, offset
251         + innerOffset, copyLength);
252     clone.position = this.position;
253     return clone;
254   }
255 
256   @Override
257   public PositionedByteRange putShort(short val) {
258     putShort(position, val);
259     position += Bytes.SIZEOF_SHORT;
260     return this;
261   }
262 
263   @Override
264   public PositionedByteRange putInt(int val) {
265     putInt(position, val);
266     position += Bytes.SIZEOF_INT;
267     return this;
268   }
269 
270   @Override
271   public PositionedByteRange putLong(long val) {
272     putLong(position, val);
273     position += Bytes.SIZEOF_LONG;
274     return this;
275   }
276 
277   @Override
278   public int putVLong(long val) {
279     int len = putVLong(position, val);
280     position += len;
281     return len;
282   }
283 
284   @Override
285   public PositionedByteRange putShort(int index, short val) {
286     // This writing is same as BB's putShort. When byte[] is wrapped in a BB and
287     // call putShort(),
288     // one can get the same result.
289     bytes[offset + index + 1] = (byte) val;
290     val >>= 8;
291     bytes[offset + index] = (byte) val;
292     clearHashCache();
293     return this;
294   }
295 
296   @Override
297   public PositionedByteRange putInt(int index, int val) {
298     // This writing is same as BB's putInt. When byte[] is wrapped in a BB and
299     // call getInt(), one
300     // can get the same result.
301     for (int i = Bytes.SIZEOF_INT - 1; i > 0; i--) {
302       bytes[offset + index + i] = (byte) val;
303       val >>>= 8;
304     }
305     bytes[offset + index] = (byte) val;
306     clearHashCache();
307     return this;
308   }
309 
310   @Override
311   public PositionedByteRange putLong(int index, long val) {
312     // This writing is same as BB's putLong. When byte[] is wrapped in a BB and
313     // call putLong(), one
314     // can get the same result.
315     for (int i = Bytes.SIZEOF_LONG - 1; i > 0; i--) {
316       bytes[offset + index + i] = (byte) val;
317       val >>>= 8;
318     }
319     bytes[offset + index] = (byte) val;
320     clearHashCache();
321     return this;
322   }
323 
324   // Copied from com.google.protobuf.CodedOutputStream v2.5.0 writeRawVarint64
325   @Override
326   public int putVLong(int index, long val) {
327     int rPos = 0;
328     while (true) {
329       if ((val & ~0x7F) == 0) {
330         bytes[offset + index + rPos] = (byte) val;
331         break;
332       } else {
333         bytes[offset + index + rPos] = (byte) ((val & 0x7F) | 0x80);
334         val >>>= 7;
335       }
336       rPos++;
337     }
338     clearHashCache();
339     return rPos + 1;
340   }
341   // end copied from protobuf
342 
343 }